@@ -16,6 +16,8 @@ use openraft::async_trait::async_trait;
16
16
use openraft:: raft:: Entry ;
17
17
use openraft:: raft:: EntryPayload ;
18
18
use openraft:: storage:: LogState ;
19
+ use openraft:: storage:: RaftLogReader ;
20
+ use openraft:: storage:: RaftSnapshotBuilder ;
19
21
use openraft:: storage:: Snapshot ;
20
22
use openraft:: AppData ;
21
23
use openraft:: AppDataResponse ;
@@ -117,35 +119,24 @@ impl MemStore {
117
119
current_snapshot,
118
120
}
119
121
}
122
+
123
+ pub async fn new_arc ( ) -> Arc < Self > {
124
+ Arc :: new ( Self :: new ( ) . await )
125
+ }
120
126
}
121
127
122
128
#[ async_trait]
123
- impl RaftStorageDebug < MemStoreStateMachine > for MemStore {
129
+ impl RaftStorageDebug < MemStoreStateMachine > for Arc < MemStore > {
124
130
/// Get a handle to the state machine for testing purposes.
125
- async fn get_state_machine ( & self ) -> MemStoreStateMachine {
131
+ async fn get_state_machine ( & mut self ) -> MemStoreStateMachine {
126
132
self . sm . write ( ) . await . clone ( )
127
133
}
128
134
}
129
135
130
136
#[ async_trait]
131
- impl RaftStorage < ClientRequest , ClientResponse > for MemStore {
132
- type SnapshotData = Cursor < Vec < u8 > > ;
133
-
134
- #[ tracing:: instrument( level = "trace" , skip( self ) ) ]
135
- async fn save_vote ( & self , vote : & Vote ) -> Result < ( ) , StorageError > {
136
- tracing:: debug!( ?vote, "save_vote" ) ;
137
- let mut h = self . vote . write ( ) . await ;
138
-
139
- * h = Some ( * vote) ;
140
- Ok ( ( ) )
141
- }
142
-
143
- async fn read_vote ( & self ) -> Result < Option < Vote > , StorageError > {
144
- Ok ( * self . vote . read ( ) . await )
145
- }
146
-
137
+ impl RaftLogReader < ClientRequest , ClientResponse > for Arc < MemStore > {
147
138
async fn try_get_log_entries < RB : RangeBounds < u64 > + Clone + Debug + Send + Sync > (
148
- & self ,
139
+ & mut self ,
149
140
range : RB ,
150
141
) -> Result < Vec < Entry < ClientRequest > > , StorageError > {
151
142
let res = {
@@ -156,7 +147,7 @@ impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
156
147
Ok ( res)
157
148
}
158
149
159
- async fn get_log_state ( & self ) -> Result < LogState , StorageError > {
150
+ async fn get_log_state ( & mut self ) -> Result < LogState , StorageError > {
160
151
let log = self . log . read ( ) . await ;
161
152
let last = log. iter ( ) . rev ( ) . next ( ) . map ( |( _, ent) | ent. log_id ) ;
162
153
@@ -172,14 +163,91 @@ impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
172
163
last_log_id : last,
173
164
} )
174
165
}
166
+ }
167
+
168
+ #[ async_trait]
169
+ impl RaftSnapshotBuilder < ClientRequest , ClientResponse , Cursor < Vec < u8 > > > for Arc < MemStore > {
170
+ #[ tracing:: instrument( level = "trace" , skip( self ) ) ]
171
+ async fn build_snapshot ( & mut self ) -> Result < Snapshot < Cursor < Vec < u8 > > > , StorageError > {
172
+ let ( data, last_applied_log) ;
173
+
174
+ {
175
+ // Serialize the data of the state machine.
176
+ let sm = self . sm . read ( ) . await ;
177
+ data = serde_json:: to_vec ( & * sm)
178
+ . map_err ( |e| StorageIOError :: new ( ErrorSubject :: StateMachine , ErrorVerb :: Read , AnyError :: new ( & e) ) ) ?;
179
+
180
+ last_applied_log = sm. last_applied_log ;
181
+ }
182
+
183
+ let last_applied_log = match last_applied_log {
184
+ None => {
185
+ panic ! ( "can not compact empty state machine" ) ;
186
+ }
187
+ Some ( x) => x,
188
+ } ;
189
+
190
+ let snapshot_size = data. len ( ) ;
191
+
192
+ let snapshot_idx = {
193
+ let mut l = self . snapshot_idx . lock ( ) . unwrap ( ) ;
194
+ * l += 1 ;
195
+ * l
196
+ } ;
197
+
198
+ let snapshot_id = format ! (
199
+ "{}-{}-{}" ,
200
+ last_applied_log. leader_id, last_applied_log. index, snapshot_idx
201
+ ) ;
202
+
203
+ let meta = SnapshotMeta {
204
+ last_log_id : last_applied_log,
205
+ snapshot_id,
206
+ } ;
207
+
208
+ let snapshot = MemStoreSnapshot {
209
+ meta : meta. clone ( ) ,
210
+ data : data. clone ( ) ,
211
+ } ;
212
+
213
+ {
214
+ let mut current_snapshot = self . current_snapshot . write ( ) . await ;
215
+ * current_snapshot = Some ( snapshot) ;
216
+ }
217
+
218
+ tracing:: info!( snapshot_size, "log compaction complete" ) ;
219
+
220
+ Ok ( Snapshot {
221
+ meta,
222
+ snapshot : Box :: new ( Cursor :: new ( data) ) ,
223
+ } )
224
+ }
225
+ }
175
226
176
- async fn last_applied_state ( & self ) -> Result < ( Option < LogId > , Option < EffectiveMembership > ) , StorageError > {
227
+ #[ async_trait]
228
+ impl RaftStorage < ClientRequest , ClientResponse > for Arc < MemStore > {
229
+ type SnapshotData = Cursor < Vec < u8 > > ;
230
+
231
+ #[ tracing:: instrument( level = "trace" , skip( self ) ) ]
232
+ async fn save_vote ( & mut self , vote : & Vote ) -> Result < ( ) , StorageError > {
233
+ tracing:: debug!( ?vote, "save_vote" ) ;
234
+ let mut h = self . vote . write ( ) . await ;
235
+
236
+ * h = Some ( * vote) ;
237
+ Ok ( ( ) )
238
+ }
239
+
240
+ async fn read_vote ( & mut self ) -> Result < Option < Vote > , StorageError > {
241
+ Ok ( * self . vote . read ( ) . await )
242
+ }
243
+
244
+ async fn last_applied_state ( & mut self ) -> Result < ( Option < LogId > , Option < EffectiveMembership > ) , StorageError > {
177
245
let sm = self . sm . read ( ) . await ;
178
246
Ok ( ( sm. last_applied_log , sm. last_membership . clone ( ) ) )
179
247
}
180
248
181
249
#[ tracing:: instrument( level = "debug" , skip( self ) ) ]
182
- async fn delete_conflict_logs_since ( & self , log_id : LogId ) -> Result < ( ) , StorageError > {
250
+ async fn delete_conflict_logs_since ( & mut self , log_id : LogId ) -> Result < ( ) , StorageError > {
183
251
tracing:: debug!( "delete_log: [{:?}, +oo)" , log_id) ;
184
252
185
253
{
@@ -195,7 +263,7 @@ impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
195
263
}
196
264
197
265
#[ tracing:: instrument( level = "debug" , skip( self ) ) ]
198
- async fn purge_logs_upto ( & self , log_id : LogId ) -> Result < ( ) , StorageError > {
266
+ async fn purge_logs_upto ( & mut self , log_id : LogId ) -> Result < ( ) , StorageError > {
199
267
tracing:: debug!( "delete_log: [{:?}, +oo)" , log_id) ;
200
268
201
269
{
@@ -217,7 +285,7 @@ impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
217
285
}
218
286
219
287
#[ tracing:: instrument( level = "trace" , skip( self , entries) ) ]
220
- async fn append_to_log ( & self , entries : & [ & Entry < ClientRequest > ] ) -> Result < ( ) , StorageError > {
288
+ async fn append_to_log ( & mut self , entries : & [ & Entry < ClientRequest > ] ) -> Result < ( ) , StorageError > {
221
289
let mut log = self . log . write ( ) . await ;
222
290
for entry in entries {
223
291
log. insert ( entry. log_id . index , ( * entry) . clone ( ) ) ;
@@ -227,7 +295,7 @@ impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
227
295
228
296
#[ tracing:: instrument( level = "trace" , skip( self , entries) ) ]
229
297
async fn apply_to_state_machine (
230
- & self ,
298
+ & mut self ,
231
299
entries : & [ & Entry < ClientRequest > ] ,
232
300
) -> Result < Vec < ClientResponse > , StorageError > {
233
301
let mut res = Vec :: with_capacity ( entries. len ( ) ) ;
@@ -262,69 +330,13 @@ impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
262
330
}
263
331
264
332
#[ tracing:: instrument( level = "trace" , skip( self ) ) ]
265
- async fn build_snapshot ( & self ) -> Result < Snapshot < Self :: SnapshotData > , StorageError > {
266
- let ( data, last_applied_log) ;
267
-
268
- {
269
- // Serialize the data of the state machine.
270
- let sm = self . sm . read ( ) . await ;
271
- data = serde_json:: to_vec ( & * sm)
272
- . map_err ( |e| StorageIOError :: new ( ErrorSubject :: StateMachine , ErrorVerb :: Read , AnyError :: new ( & e) ) ) ?;
273
-
274
- last_applied_log = sm. last_applied_log ;
275
- }
276
-
277
- let last_applied_log = match last_applied_log {
278
- None => {
279
- panic ! ( "can not compact empty state machine" ) ;
280
- }
281
- Some ( x) => x,
282
- } ;
283
-
284
- let snapshot_size = data. len ( ) ;
285
-
286
- let snapshot_idx = {
287
- let mut l = self . snapshot_idx . lock ( ) . unwrap ( ) ;
288
- * l += 1 ;
289
- * l
290
- } ;
291
-
292
- let snapshot_id = format ! (
293
- "{}-{}-{}" ,
294
- last_applied_log. leader_id, last_applied_log. index, snapshot_idx
295
- ) ;
296
-
297
- let meta = SnapshotMeta {
298
- last_log_id : last_applied_log,
299
- snapshot_id,
300
- } ;
301
-
302
- let snapshot = MemStoreSnapshot {
303
- meta : meta. clone ( ) ,
304
- data : data. clone ( ) ,
305
- } ;
306
-
307
- {
308
- let mut current_snapshot = self . current_snapshot . write ( ) . await ;
309
- * current_snapshot = Some ( snapshot) ;
310
- }
311
-
312
- tracing:: info!( snapshot_size, "log compaction complete" ) ;
313
-
314
- Ok ( Snapshot {
315
- meta,
316
- snapshot : Box :: new ( Cursor :: new ( data) ) ,
317
- } )
318
- }
319
-
320
- #[ tracing:: instrument( level = "trace" , skip( self ) ) ]
321
- async fn begin_receiving_snapshot ( & self ) -> Result < Box < Self :: SnapshotData > , StorageError > {
333
+ async fn begin_receiving_snapshot ( & mut self ) -> Result < Box < Self :: SnapshotData > , StorageError > {
322
334
Ok ( Box :: new ( Cursor :: new ( Vec :: new ( ) ) ) )
323
335
}
324
336
325
337
#[ tracing:: instrument( level = "trace" , skip( self , snapshot) ) ]
326
338
async fn install_snapshot (
327
- & self ,
339
+ & mut self ,
328
340
meta : & SnapshotMeta ,
329
341
snapshot : Box < Self :: SnapshotData > ,
330
342
) -> Result < StateMachineChanges , StorageError > {
@@ -368,7 +380,7 @@ impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
368
380
}
369
381
370
382
#[ tracing:: instrument( level = "trace" , skip( self ) ) ]
371
- async fn get_current_snapshot ( & self ) -> Result < Option < Snapshot < Self :: SnapshotData > > , StorageError > {
383
+ async fn get_current_snapshot ( & mut self ) -> Result < Option < Snapshot < Self :: SnapshotData > > , StorageError > {
372
384
match & * self . current_snapshot . read ( ) . await {
373
385
Some ( snapshot) => {
374
386
let data = snapshot. data . clone ( ) ;
@@ -380,4 +392,15 @@ impl RaftStorage<ClientRequest, ClientResponse> for MemStore {
380
392
None => Ok ( None ) ,
381
393
}
382
394
}
395
+
396
+ type LogReader = Self ;
397
+ type SnapshotBuilder = Self ;
398
+
399
+ async fn get_log_reader ( & mut self ) -> Self :: LogReader {
400
+ self . clone ( )
401
+ }
402
+
403
+ async fn get_snapshot_builder ( & mut self ) -> Self :: SnapshotBuilder {
404
+ self . clone ( )
405
+ }
383
406
}
0 commit comments