55
55
import org .apache .iotdb .rpc .TSStatusCode ;
56
56
57
57
import org .apache .thrift .TException ;
58
- import org .apache .thrift .async .AsyncMethodCallback ;
59
58
import org .slf4j .Logger ;
60
59
import org .slf4j .LoggerFactory ;
61
60
62
61
import java .util .Collections ;
63
62
import java .util .stream .Collectors ;
64
63
65
- public class IoTConsensusRPCServiceProcessor implements IoTConsensusIService .AsyncIface {
64
+ public class IoTConsensusRPCServiceProcessor implements IoTConsensusIService .Iface {
66
65
67
66
private static final Logger LOGGER =
68
67
LoggerFactory .getLogger (IoTConsensusRPCServiceProcessor .class );
@@ -74,98 +73,85 @@ public IoTConsensusRPCServiceProcessor(IoTConsensus consensus) {
74
73
}
75
74
76
75
@ Override
77
- public void syncLogEntries (
78
- TSyncLogEntriesReq req , AsyncMethodCallback <TSyncLogEntriesRes > resultHandler ) {
79
- try {
80
- ConsensusGroupId groupId =
81
- ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
82
- IoTConsensusServerImpl impl = consensus .getImpl (groupId );
83
- if (impl == null ) {
84
- String message =
85
- String .format (
86
- "unexpected consensusGroupId %s for TSyncLogEntriesReq which size is %s" ,
87
- groupId , req .getLogEntries ().size ());
88
- LOGGER .error (message );
89
- TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
90
- status .setMessage (message );
91
- resultHandler .onComplete (new TSyncLogEntriesRes (Collections .singletonList (status )));
92
- return ;
93
- }
94
- if (impl .isReadOnly ()) {
95
- String message = "fail to sync logEntries because system is read-only." ;
96
- LOGGER .error (message );
97
- TSStatus status = new TSStatus (TSStatusCode .SYSTEM_READ_ONLY .getStatusCode ());
98
- status .setMessage (message );
99
- resultHandler .onComplete (new TSyncLogEntriesRes (Collections .singletonList (status )));
100
- return ;
101
- }
102
- if (!impl .isActive ()) {
103
- TSStatus status = new TSStatus (TSStatusCode .WRITE_PROCESS_REJECT .getStatusCode ());
104
- status .setMessage ("peer is inactive and not ready to receive sync log request" );
105
- resultHandler .onComplete (new TSyncLogEntriesRes (Collections .singletonList (status )));
106
- return ;
107
- }
108
- BatchIndexedConsensusRequest logEntriesInThisBatch =
109
- new BatchIndexedConsensusRequest (req .peerId );
110
- // We use synchronized to ensure atomicity of executing multiple logs
111
- for (TLogEntry entry : req .getLogEntries ()) {
112
- logEntriesInThisBatch .add (
113
- impl .buildIndexedConsensusRequestForRemoteRequest (
114
- entry .getSearchIndex (),
115
- entry .getData ().stream ()
116
- .map (
117
- entry .isFromWAL ()
118
- ? IoTConsensusRequest ::new
119
- : ByteBufferConsensusRequest ::new )
120
- .collect (Collectors .toList ())));
121
- }
122
- long buildRequestTime = System .nanoTime ();
123
- IConsensusRequest deserializedRequest =
124
- impl .getStateMachine ().deserializeRequest (logEntriesInThisBatch );
125
- impl .getIoTConsensusServerMetrics ()
126
- .recordDeserializeCost (System .nanoTime () - buildRequestTime );
127
- TSStatus writeStatus =
128
- impl .syncLog (logEntriesInThisBatch .getSourcePeerId (), deserializedRequest );
129
- LOGGER .debug (
130
- "execute TSyncLogEntriesReq for {} with result {}" ,
131
- req .consensusGroupId ,
132
- writeStatus .subStatus );
133
- resultHandler .onComplete (new TSyncLogEntriesRes (writeStatus .subStatus ));
134
- } catch (Exception e ) {
135
- resultHandler .onError (e );
76
+ public TSyncLogEntriesRes syncLogEntries (TSyncLogEntriesReq req ) {
77
+ ConsensusGroupId groupId =
78
+ ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
79
+ IoTConsensusServerImpl impl = consensus .getImpl (groupId );
80
+ if (impl == null ) {
81
+ String message =
82
+ String .format (
83
+ "unexpected consensusGroupId %s for TSyncLogEntriesReq which size is %s" ,
84
+ groupId , req .getLogEntries ().size ());
85
+ LOGGER .error (message );
86
+ TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
87
+ status .setMessage (message );
88
+ return new TSyncLogEntriesRes (Collections .singletonList (status ));
136
89
}
90
+ if (impl .isReadOnly ()) {
91
+ String message = "fail to sync logEntries because system is read-only." ;
92
+ LOGGER .error (message );
93
+ TSStatus status = new TSStatus (TSStatusCode .SYSTEM_READ_ONLY .getStatusCode ());
94
+ status .setMessage (message );
95
+ return new TSyncLogEntriesRes (Collections .singletonList (status ));
96
+ }
97
+ if (!impl .isActive ()) {
98
+ TSStatus status = new TSStatus (TSStatusCode .WRITE_PROCESS_REJECT .getStatusCode ());
99
+ status .setMessage ("peer is inactive and not ready to receive sync log request" );
100
+ return new TSyncLogEntriesRes (Collections .singletonList (status ));
101
+ }
102
+ BatchIndexedConsensusRequest logEntriesInThisBatch =
103
+ new BatchIndexedConsensusRequest (req .peerId );
104
+ // We use synchronized to ensure atomicity of executing multiple logs
105
+ for (TLogEntry entry : req .getLogEntries ()) {
106
+ logEntriesInThisBatch .add (
107
+ impl .buildIndexedConsensusRequestForRemoteRequest (
108
+ entry .getSearchIndex (),
109
+ entry .getData ().stream ()
110
+ .map (
111
+ entry .isFromWAL ()
112
+ ? IoTConsensusRequest ::new
113
+ : ByteBufferConsensusRequest ::new )
114
+ .collect (Collectors .toList ())));
115
+ }
116
+ long buildRequestTime = System .nanoTime ();
117
+ IConsensusRequest deserializedRequest =
118
+ impl .getStateMachine ().deserializeRequest (logEntriesInThisBatch );
119
+ impl .getIoTConsensusServerMetrics ().recordDeserializeCost (System .nanoTime () - buildRequestTime );
120
+ TSStatus writeStatus =
121
+ impl .syncLog (logEntriesInThisBatch .getSourcePeerId (), deserializedRequest );
122
+ LOGGER .debug (
123
+ "execute TSyncLogEntriesReq for {} with result {}" ,
124
+ req .consensusGroupId ,
125
+ writeStatus .subStatus );
126
+ return new TSyncLogEntriesRes (writeStatus .subStatus );
137
127
}
138
128
139
129
@ Override
140
- public void inactivatePeer (
141
- TInactivatePeerReq req , AsyncMethodCallback <TInactivatePeerRes > resultHandler )
142
- throws TException {
130
+ public TInactivatePeerRes inactivatePeer (TInactivatePeerReq req ) throws TException {
143
131
if (req .isForDeletionPurpose ()) {
144
132
KillPoint .setKillPoint (IoTConsensusInactivatePeerKillPoints .BEFORE_INACTIVATE );
145
133
}
146
134
ConsensusGroupId groupId =
147
135
ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
148
136
IoTConsensusServerImpl impl = consensus .getImpl (groupId );
137
+
149
138
if (impl == null ) {
150
139
String message =
151
140
String .format ("unexpected consensusGroupId %s for inactivatePeer request" , groupId );
152
141
LOGGER .error (message );
153
142
TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
154
143
status .setMessage (message );
155
- resultHandler .onComplete (new TInactivatePeerRes (status ));
156
- return ;
144
+ return new TInactivatePeerRes (status );
157
145
}
158
146
impl .setActive (false );
159
- resultHandler .onComplete (
160
- new TInactivatePeerRes (new TSStatus (TSStatusCode .SUCCESS_STATUS .getStatusCode ())));
161
147
if (req .isForDeletionPurpose ()) {
162
148
KillPoint .setKillPoint (IoTConsensusInactivatePeerKillPoints .AFTER_INACTIVATE );
163
149
}
150
+ return new TInactivatePeerRes (new TSStatus (TSStatusCode .SUCCESS_STATUS .getStatusCode ()));
164
151
}
165
152
166
153
@ Override
167
- public void activatePeer (
168
- TActivatePeerReq req , AsyncMethodCallback <TActivatePeerRes > resultHandler ) throws TException {
154
+ public TActivatePeerRes activatePeer (TActivatePeerReq req ) throws TException {
169
155
ConsensusGroupId groupId =
170
156
ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
171
157
IoTConsensusServerImpl impl = consensus .getImpl (groupId );
@@ -175,18 +161,15 @@ public void activatePeer(
175
161
LOGGER .error (message );
176
162
TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
177
163
status .setMessage (message );
178
- resultHandler .onComplete (new TActivatePeerRes (status ));
179
- return ;
164
+ return new TActivatePeerRes (status );
180
165
}
181
166
KillPoint .setKillPoint (DataNodeKillPoints .DESTINATION_ADD_PEER_DONE );
182
167
impl .setActive (true );
183
- resultHandler .onComplete (
184
- new TActivatePeerRes (new TSStatus (TSStatusCode .SUCCESS_STATUS .getStatusCode ())));
168
+ return new TActivatePeerRes (new TSStatus (TSStatusCode .SUCCESS_STATUS .getStatusCode ()));
185
169
}
186
170
187
171
@ Override
188
- public void buildSyncLogChannel (
189
- TBuildSyncLogChannelReq req , AsyncMethodCallback <TBuildSyncLogChannelRes > resultHandler )
172
+ public TBuildSyncLogChannelRes buildSyncLogChannel (TBuildSyncLogChannelReq req )
190
173
throws TException {
191
174
ConsensusGroupId groupId =
192
175
ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
@@ -197,8 +180,7 @@ public void buildSyncLogChannel(
197
180
LOGGER .error (message );
198
181
TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
199
182
status .setMessage (message );
200
- resultHandler .onComplete (new TBuildSyncLogChannelRes (status ));
201
- return ;
183
+ return new TBuildSyncLogChannelRes (status );
202
184
}
203
185
TSStatus responseStatus ;
204
186
try {
@@ -208,12 +190,11 @@ public void buildSyncLogChannel(
208
190
responseStatus = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
209
191
responseStatus .setMessage (e .getMessage ());
210
192
}
211
- resultHandler . onComplete ( new TBuildSyncLogChannelRes (responseStatus ) );
193
+ return new TBuildSyncLogChannelRes (responseStatus );
212
194
}
213
195
214
196
@ Override
215
- public void removeSyncLogChannel (
216
- TRemoveSyncLogChannelReq req , AsyncMethodCallback <TRemoveSyncLogChannelRes > resultHandler )
197
+ public TRemoveSyncLogChannelRes removeSyncLogChannel (TRemoveSyncLogChannelReq req )
217
198
throws TException {
218
199
ConsensusGroupId groupId =
219
200
ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
@@ -224,8 +205,7 @@ public void removeSyncLogChannel(
224
205
LOGGER .error (message );
225
206
TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
226
207
status .setMessage (message );
227
- resultHandler .onComplete (new TRemoveSyncLogChannelRes (status ));
228
- return ;
208
+ return new TRemoveSyncLogChannelRes (status );
229
209
}
230
210
TSStatus responseStatus ;
231
211
try {
@@ -235,12 +215,11 @@ public void removeSyncLogChannel(
235
215
responseStatus = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
236
216
responseStatus .setMessage (e .getMessage ());
237
217
}
238
- resultHandler . onComplete ( new TRemoveSyncLogChannelRes (responseStatus ) );
218
+ return new TRemoveSyncLogChannelRes (responseStatus );
239
219
}
240
220
241
221
@ Override
242
- public void waitSyncLogComplete (
243
- TWaitSyncLogCompleteReq req , AsyncMethodCallback <TWaitSyncLogCompleteRes > resultHandler )
222
+ public TWaitSyncLogCompleteRes waitSyncLogComplete (TWaitSyncLogCompleteReq req )
244
223
throws TException {
245
224
ConsensusGroupId groupId =
246
225
ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
@@ -251,18 +230,15 @@ public void waitSyncLogComplete(
251
230
LOGGER .error (message );
252
231
TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
253
232
status .setMessage (message );
254
- resultHandler .onComplete (new TWaitSyncLogCompleteRes (true , 0 , 0 ));
255
- return ;
233
+ return new TWaitSyncLogCompleteRes (true , 0 , 0 );
256
234
}
257
235
long searchIndex = impl .getSearchIndex ();
258
236
long safeIndex = impl .getMinSyncIndex ();
259
- resultHandler .onComplete (
260
- new TWaitSyncLogCompleteRes (searchIndex == safeIndex , searchIndex , safeIndex ));
237
+ return new TWaitSyncLogCompleteRes (searchIndex == safeIndex , searchIndex , safeIndex );
261
238
}
262
239
263
240
@ Override
264
- public void sendSnapshotFragment (
265
- TSendSnapshotFragmentReq req , AsyncMethodCallback <TSendSnapshotFragmentRes > resultHandler )
241
+ public TSendSnapshotFragmentRes sendSnapshotFragment (TSendSnapshotFragmentReq req )
266
242
throws TException {
267
243
ConsensusGroupId groupId =
268
244
ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
@@ -273,8 +249,7 @@ public void sendSnapshotFragment(
273
249
LOGGER .error (message );
274
250
TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
275
251
status .setMessage (message );
276
- resultHandler .onComplete (new TSendSnapshotFragmentRes (status ));
277
- return ;
252
+ return new TSendSnapshotFragmentRes (status );
278
253
}
279
254
TSStatus responseStatus ;
280
255
try {
@@ -284,12 +259,11 @@ public void sendSnapshotFragment(
284
259
responseStatus = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
285
260
responseStatus .setMessage (e .getMessage ());
286
261
}
287
- resultHandler . onComplete ( new TSendSnapshotFragmentRes (responseStatus ) );
262
+ return new TSendSnapshotFragmentRes (responseStatus );
288
263
}
289
264
290
265
@ Override
291
- public void triggerSnapshotLoad (
292
- TTriggerSnapshotLoadReq req , AsyncMethodCallback <TTriggerSnapshotLoadRes > resultHandler )
266
+ public TTriggerSnapshotLoadRes triggerSnapshotLoad (TTriggerSnapshotLoadReq req )
293
267
throws TException {
294
268
ConsensusGroupId groupId =
295
269
ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
@@ -300,20 +274,16 @@ public void triggerSnapshotLoad(
300
274
LOGGER .error (message );
301
275
TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
302
276
status .setMessage (message );
303
- resultHandler .onComplete (new TTriggerSnapshotLoadRes (status ));
304
- return ;
277
+ return new TTriggerSnapshotLoadRes (status );
305
278
}
306
279
impl .loadSnapshot (req .snapshotId );
307
280
KillPoint .setKillPoint (DataNodeKillPoints .DESTINATION_ADD_PEER_TRANSITION );
308
- resultHandler .onComplete (
309
- new TTriggerSnapshotLoadRes (new TSStatus (TSStatusCode .SUCCESS_STATUS .getStatusCode ())));
281
+ return new TTriggerSnapshotLoadRes (new TSStatus (TSStatusCode .SUCCESS_STATUS .getStatusCode ()));
310
282
}
311
283
312
284
@ Override
313
- public void cleanupTransferredSnapshot (
314
- TCleanupTransferredSnapshotReq req ,
315
- AsyncMethodCallback <TCleanupTransferredSnapshotRes > resultHandler )
316
- throws TException {
285
+ public TCleanupTransferredSnapshotRes cleanupTransferredSnapshot (
286
+ TCleanupTransferredSnapshotReq req ) throws TException {
317
287
ConsensusGroupId groupId =
318
288
ConsensusGroupId .Factory .createFromTConsensusGroupId (req .getConsensusGroupId ());
319
289
IoTConsensusServerImpl impl = consensus .getImpl (groupId );
@@ -323,8 +293,7 @@ public void cleanupTransferredSnapshot(
323
293
LOGGER .error (message );
324
294
TSStatus status = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
325
295
status .setMessage (message );
326
- resultHandler .onComplete (new TCleanupTransferredSnapshotRes (status ));
327
- return ;
296
+ return new TCleanupTransferredSnapshotRes (status );
328
297
}
329
298
TSStatus responseStatus ;
330
299
try {
@@ -335,7 +304,7 @@ public void cleanupTransferredSnapshot(
335
304
responseStatus = new TSStatus (TSStatusCode .INTERNAL_SERVER_ERROR .getStatusCode ());
336
305
responseStatus .setMessage (e .getMessage ());
337
306
}
338
- resultHandler . onComplete ( new TCleanupTransferredSnapshotRes (responseStatus ) );
307
+ return new TCleanupTransferredSnapshotRes (responseStatus );
339
308
}
340
309
341
310
public void handleClientExit () {}
0 commit comments