-
Notifications
You must be signed in to change notification settings - Fork 90
/
RaftGroupServiceImpl.java
766 lines (630 loc) · 28.1 KB
/
RaftGroupServiceImpl.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.ignite.internal.raft;
import static java.lang.System.currentTimeMillis;
import static java.util.concurrent.ThreadLocalRandom.current;
import static java.util.stream.Collectors.toList;
import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.AddLearnersRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.AddPeerRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.AddPeerResponse;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.ChangePeersAsyncRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.ChangePeersAsyncResponse;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.ChangePeersRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.ChangePeersResponse;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.GetLeaderRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.GetLeaderResponse;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.GetPeersRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.GetPeersResponse;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.LearnersOpResponse;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.RemoveLearnersRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.RemovePeerRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.RemovePeerResponse;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.ResetLearnersRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.SnapshotRequest;
import static org.apache.ignite.raft.jraft.rpc.CliRequests.TransferLeaderRequest;
import java.io.IOException;
import java.net.ConnectException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
import org.apache.ignite.internal.raft.service.LeaderMetadata;
import org.apache.ignite.internal.raft.service.LeaderWithTerm;
import org.apache.ignite.internal.raft.service.RaftGroupService;
import org.apache.ignite.internal.replicator.ReplicationGroupId;
import org.apache.ignite.internal.tostring.S;
import org.apache.ignite.internal.util.IgniteSpinBusyLock;
import org.apache.ignite.lang.IgniteException;
import org.apache.ignite.lang.IgniteInternalException;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.network.ClusterService;
import org.apache.ignite.network.NetworkMessage;
import org.apache.ignite.raft.jraft.RaftMessagesFactory;
import org.apache.ignite.raft.jraft.entity.PeerId;
import org.apache.ignite.raft.jraft.error.RaftError;
import org.apache.ignite.raft.jraft.rpc.ActionRequest;
import org.apache.ignite.raft.jraft.rpc.ActionResponse;
import org.apache.ignite.raft.jraft.rpc.RpcRequests;
import org.apache.ignite.raft.jraft.rpc.RpcRequests.ErrorResponse;
import org.apache.ignite.raft.jraft.rpc.RpcRequests.ReadLeaderMetadataResponse;
import org.apache.ignite.raft.jraft.rpc.RpcRequests.SMErrorResponse;
import org.apache.ignite.raft.jraft.rpc.impl.RaftException;
import org.apache.ignite.raft.jraft.rpc.impl.SMCompactedThrowable;
import org.apache.ignite.raft.jraft.rpc.impl.SMFullThrowable;
import org.apache.ignite.raft.jraft.rpc.impl.SMThrowable;
import org.jetbrains.annotations.Nullable;
/**
* The implementation of {@link RaftGroupService}.
*/
public class RaftGroupServiceImpl implements RaftGroupService {
/** The logger. */
private static final IgniteLogger LOG = Loggers.forClass(RaftGroupServiceImpl.class);
private final String groupId;
private final ReplicationGroupId realGroupId;
private final RaftMessagesFactory factory;
private final RaftConfiguration configuration;
@Nullable
private volatile Peer leader;
private volatile List<Peer> peers;
private volatile List<Peer> learners;
private final ClusterService cluster;
/** Executor for scheduling retries of {@link RaftGroupServiceImpl#sendWithRetry} invocations. */
private final ScheduledExecutorService executor;
/** Busy lock. */
private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
/**
* Constructor.
*
* @param groupId Group id.
* @param cluster A cluster.
* @param factory A message factory.
* @param configuration Raft configuration.
* @param membersConfiguration Raft members configuration.
* @param leader Group leader.
* @param executor Executor for retrying requests.
*/
private RaftGroupServiceImpl(
ReplicationGroupId groupId,
ClusterService cluster,
RaftMessagesFactory factory,
RaftConfiguration configuration,
PeersAndLearners membersConfiguration,
@Nullable Peer leader,
ScheduledExecutorService executor
) {
this.cluster = cluster;
this.configuration = configuration;
this.peers = List.copyOf(membersConfiguration.peers());
this.learners = List.copyOf(membersConfiguration.learners());
this.factory = factory;
this.groupId = groupId.toString();
this.realGroupId = groupId;
this.leader = leader;
this.executor = executor;
}
/**
* Starts raft group service.
*
* @param groupId Raft group id.
* @param cluster Cluster service.
* @param factory Message factory.
* @param configuration Raft configuration.
* @param membersConfiguration Raft members configuration.
* @param getLeader {@code True} to get the group's leader upon service creation.
* @param executor Executor for retrying requests.
* @return Future representing pending completion of the operation.
*/
public static CompletableFuture<RaftGroupService> start(
ReplicationGroupId groupId,
ClusterService cluster,
RaftMessagesFactory factory,
RaftConfiguration configuration,
PeersAndLearners membersConfiguration,
boolean getLeader,
ScheduledExecutorService executor
) {
var service = new RaftGroupServiceImpl(
groupId,
cluster,
factory,
configuration,
membersConfiguration,
null,
executor
);
if (!getLeader) {
return CompletableFuture.completedFuture(service);
}
return service.refreshLeader().handle((unused, throwable) -> {
if (throwable != null) {
if (throwable.getCause() instanceof TimeoutException) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to refresh a leader [groupId={}]", groupId);
}
} else {
if (LOG.isWarnEnabled()) {
LOG.warn("Failed to refresh a leader [groupId={}]", throwable, groupId);
}
}
}
return service;
});
}
@Override
public ReplicationGroupId groupId() {
return realGroupId;
}
@Override
public Peer leader() {
return leader;
}
@Override
public List<Peer> peers() {
return peers;
}
@Override
public List<Peer> learners() {
return learners;
}
@Override
public CompletableFuture<Void> refreshLeader() {
Function<Peer, GetLeaderRequest> requestFactory = targetPeer -> factory.getLeaderRequest()
.peerId(peerId(targetPeer))
.groupId(groupId)
.build();
return this.<GetLeaderResponse>sendWithRetry(randomNode(), requestFactory)
.thenAccept(resp -> this.leader = parsePeer(resp.leaderId()));
}
@Override
public CompletableFuture<LeaderWithTerm> refreshAndGetLeaderWithTerm() {
Function<Peer, GetLeaderRequest> requestFactory = targetPeer -> factory.getLeaderRequest()
.peerId(peerId(targetPeer))
.groupId(groupId)
.build();
return this.<GetLeaderResponse>sendWithRetry(randomNode(), requestFactory)
.thenApply(resp -> {
Peer respLeader = parsePeer(resp.leaderId());
this.leader = respLeader;
return new LeaderWithTerm(respLeader, resp.currentTerm());
});
}
@Override
public CompletableFuture<Void> refreshMembers(boolean onlyAlive) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> refreshMembers(onlyAlive));
}
Function<Peer, GetPeersRequest> requestFactory = targetPeer -> factory.getPeersRequest()
.leaderId(peerId(targetPeer))
.onlyAlive(onlyAlive)
.groupId(groupId)
.build();
return this.<GetPeersResponse>sendWithRetry(leader, requestFactory)
.thenAccept(resp -> {
this.peers = parsePeerList(resp.peersList());
this.learners = parsePeerList(resp.learnersList());
});
}
@Override
public CompletableFuture<Void> addPeer(Peer peer) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> addPeer(peer));
}
Function<Peer, AddPeerRequest> requestFactory = targetPeer -> factory.addPeerRequest()
.leaderId(peerId(targetPeer))
.groupId(groupId)
.peerId(peerId(peer))
.build();
return this.<AddPeerResponse>sendWithRetry(leader, requestFactory)
.thenAccept(resp -> this.peers = parsePeerList(resp.newPeersList()));
}
@Override
public CompletableFuture<Void> removePeer(Peer peer) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> removePeer(peer));
}
Function<Peer, RemovePeerRequest> requestFactory = targetPeer -> factory.removePeerRequest()
.leaderId(peerId(targetPeer))
.groupId(groupId)
.peerId(peerId(peer))
.build();
return this.<RemovePeerResponse>sendWithRetry(leader, requestFactory)
.thenAccept(resp -> this.peers = parsePeerList(resp.newPeersList()));
}
@Override
public CompletableFuture<Void> changePeers(Collection<Peer> peers) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> changePeers(peers));
}
Function<Peer, ChangePeersRequest> requestFactory = targetPeer -> factory.changePeersRequest()
.leaderId(peerId(targetPeer))
.groupId(groupId)
.newPeersList(peerIds(peers))
.build();
return this.<ChangePeersResponse>sendWithRetry(leader, requestFactory)
.thenAccept(resp -> this.peers = parsePeerList(resp.newPeersList()));
}
@Override
public CompletableFuture<Void> changePeersAsync(PeersAndLearners peersAndLearners, long term) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> changePeersAsync(peersAndLearners, term));
}
Function<Peer, ChangePeersAsyncRequest> requestFactory = targetPeer -> factory.changePeersAsyncRequest()
.leaderId(peerId(targetPeer))
.groupId(groupId)
.term(term)
.newPeersList(peerIds(peersAndLearners.peers()))
.newLearnersList(peerIds(peersAndLearners.learners()))
.build();
LOG.info("Sending changePeersAsync request for group={} to peers={} and learners={} with leader term={}",
groupId, peersAndLearners.peers(), peersAndLearners.learners(), term);
return this.<ChangePeersAsyncResponse>sendWithRetry(leader, requestFactory)
.thenAccept(resp -> {
// We expect that all raft related errors will be handled by sendWithRetry, means that
// such responses will initiate a retrying of the original request.
assert !(resp instanceof RpcRequests.ErrorResponse);
});
}
@Override
public CompletableFuture<Void> addLearners(Collection<Peer> learners) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> addLearners(learners));
}
Function<Peer, AddLearnersRequest> requestFactory = targetPeer -> factory.addLearnersRequest()
.leaderId(peerId(targetPeer))
.groupId(groupId)
.learnersList(peerIds(learners))
.build();
return this.<LearnersOpResponse>sendWithRetry(leader, requestFactory)
.thenAccept(resp -> this.learners = parsePeerList(resp.newLearnersList()));
}
@Override
public CompletableFuture<Void> removeLearners(Collection<Peer> learners) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> removeLearners(learners));
}
Function<Peer, RemoveLearnersRequest> requestFactory = targetPeer -> factory.removeLearnersRequest()
.leaderId(peerId(targetPeer))
.groupId(groupId)
.learnersList(peerIds(learners))
.build();
return this.<LearnersOpResponse>sendWithRetry(leader, requestFactory)
.thenAccept(resp -> this.learners = parsePeerList(resp.newLearnersList()));
}
@Override
public CompletableFuture<Void> resetLearners(Collection<Peer> learners) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> resetLearners(learners));
}
Function<Peer, ResetLearnersRequest> requestFactory = targetPeer -> factory.resetLearnersRequest()
.leaderId(peerId(targetPeer))
.groupId(groupId)
.learnersList(peerIds(learners))
.build();
return this.<LearnersOpResponse>sendWithRetry(leader, requestFactory)
.thenAccept(resp -> this.learners = parsePeerList(resp.newLearnersList()));
}
@Override
public CompletableFuture<Void> snapshot(Peer peer) {
SnapshotRequest req = factory.snapshotRequest()
.peerId(peerId(peer))
.groupId(groupId)
.build();
// Disable the timeout for a snapshot request.
return resolvePeer(peer)
.thenCompose(node -> cluster.messagingService().invoke(node, req, Integer.MAX_VALUE))
.thenAccept(resp -> {
if (resp != null) {
RpcRequests.ErrorResponse resp0 = (RpcRequests.ErrorResponse) resp;
if (resp0.errorCode() != RaftError.SUCCESS.getNumber()) {
var ex = new RaftException(RaftError.forNumber(resp0.errorCode()), resp0.errorMsg());
throw new CompletionException(ex);
}
}
});
}
@Override
public CompletableFuture<Void> transferLeadership(Peer newLeader) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> transferLeadership(newLeader));
}
Function<Peer, TransferLeaderRequest> requestFactory = targetPeer -> factory.transferLeaderRequest()
.groupId(groupId)
.leaderId(peerId(targetPeer))
.peerId(peerId(newLeader))
.build();
return sendWithRetry(leader, requestFactory)
.thenRun(() -> this.leader = newLeader);
}
@Override
public <R> CompletableFuture<R> run(Command cmd) {
Peer leader = this.leader;
if (leader == null) {
return refreshLeader().thenCompose(res -> run(cmd));
}
Function<Peer, ActionRequest> requestFactory = targetPeer -> factory.actionRequest()
.command(cmd)
.groupId(groupId)
.readOnlySafe(true)
.build();
return this.<ActionResponse>sendWithRetry(leader, requestFactory)
.thenApply(resp -> (R) resp.result());
}
// TODO: IGNITE-18636 Shutdown raft services on components' stop.
@Override
public void shutdown() {
busyLock.block();
}
@Override
public CompletableFuture<LeaderMetadata> readLeaderMetadata() {
Function<Peer, ? extends NetworkMessage> requestFactory = p -> factory.readLeaderMetadataRequest()
.groupId(groupId)
.peerId(p.consistentId())
.build();
Peer leader = leader();
Peer node = leader == null ? randomNode() : leader;
return this.<ReadLeaderMetadataResponse>sendWithRetry(node, requestFactory)
.thenApply(resp -> new LeaderMetadata(parsePeer(resp.leaderId()), resp.currentTerm(), resp.index()));
}
@Override
public ClusterService clusterService() {
return cluster;
}
private <R extends NetworkMessage> CompletableFuture<R> sendWithRetry(
Peer peer, Function<Peer, ? extends NetworkMessage> requestFactory
) {
var future = new CompletableFuture<R>();
sendWithRetry(peer, requestFactory, currentTimeMillis() + configuration.retryTimeout().value(), future);
return future;
}
/**
* Retries a request until success or timeout.
*
* @param peer Initial target peer, request can be sent to a random peer if the target peer is unavailable.
* @param requestFactory Factory for creating requests to the target peer.
* @param stopTime Stop time.
* @param fut The future.
* @param <R> Response type.
*/
private <R extends NetworkMessage> void sendWithRetry(
Peer peer, Function<Peer, ? extends NetworkMessage> requestFactory, long stopTime, CompletableFuture<R> fut
) {
if (!busyLock.enterBusy()) {
fut.cancel(true);
return;
}
try {
if (currentTimeMillis() >= stopTime) {
fut.completeExceptionally(new TimeoutException());
return;
}
NetworkMessage request = requestFactory.apply(peer);
resolvePeer(peer)
.thenCompose(node -> cluster.messagingService().invoke(node, request, configuration.responseTimeout().value()))
.whenComplete((resp, err) -> {
if (LOG.isTraceEnabled()) {
LOG.trace("sendWithRetry resp={} from={} to={} err={}",
S.toString(resp),
cluster.topologyService().localMember().address(),
peer.consistentId(),
err == null ? null : err.getMessage());
}
if (err != null) {
handleThrowable(err, peer, request, requestFactory, stopTime, fut);
} else if (resp instanceof ErrorResponse) {
handleErrorResponse((ErrorResponse) resp, peer, request, requestFactory, stopTime, fut);
} else if (resp instanceof SMErrorResponse) {
handleSmErrorResponse((SMErrorResponse) resp, fut);
} else {
leader = peer; // The OK response was received from a leader.
fut.complete((R) resp);
}
});
} finally {
busyLock.leaveBusy();
}
}
private void handleThrowable(
Throwable err,
Peer peer,
NetworkMessage sentRequest,
Function<Peer, ? extends NetworkMessage> requestFactory,
long stopTime,
CompletableFuture<? extends NetworkMessage> fut
) {
if (recoverable(err)) {
LOG.warn(
"Recoverable error during the request type={} occurred (will be retried on the randomly selected node): ",
err, sentRequest.getClass().getSimpleName()
);
scheduleRetry(() -> sendWithRetry(randomNode(peer), requestFactory, stopTime, fut));
} else {
fut.completeExceptionally(err);
}
}
private void handleErrorResponse(
ErrorResponse resp,
Peer peer,
NetworkMessage sentRequest,
Function<Peer, ? extends NetworkMessage> requestFactory,
long stopTime,
CompletableFuture<? extends NetworkMessage> fut
) {
RaftError error = RaftError.forNumber(resp.errorCode());
switch (error) {
case SUCCESS:
leader = peer; // The OK response was received from a leader.
fut.complete(null); // Void response.
break;
case EBUSY:
case EAGAIN:
scheduleRetry(() -> sendWithRetry(peer, requestFactory, stopTime, fut));
break;
case ENOENT:
scheduleRetry(() -> {
// If changing peers or requesting a leader and something is not found
// probably target peer is doing rebalancing, try another peer.
if (sentRequest instanceof GetLeaderRequest || sentRequest instanceof ChangePeersAsyncRequest) {
sendWithRetry(randomNode(peer), requestFactory, stopTime, fut);
} else {
sendWithRetry(peer, requestFactory, stopTime, fut);
}
});
break;
case EPERM:
// TODO: IGNITE-15706
case UNKNOWN:
case EINTERNAL:
if (resp.leaderId() == null) {
scheduleRetry(() -> sendWithRetry(randomNode(peer), requestFactory, stopTime, fut));
} else {
leader = parsePeer(resp.leaderId()); // Update a leader.
scheduleRetry(() -> sendWithRetry(leader, requestFactory, stopTime, fut));
}
break;
default:
fut.completeExceptionally(new RaftException(error, resp.errorMsg()));
break;
}
}
private static void handleSmErrorResponse(SMErrorResponse resp, CompletableFuture<? extends NetworkMessage> fut) {
SMThrowable th = resp.error();
if (th instanceof SMCompactedThrowable) {
SMCompactedThrowable compactedThrowable = (SMCompactedThrowable) th;
try {
Throwable restoredTh = (Throwable) Class.forName(compactedThrowable.throwableClassName())
.getConstructor(String.class)
.newInstance(compactedThrowable.throwableMessage());
fut.completeExceptionally(restoredTh);
} catch (Exception e) {
LOG.warn("Cannot restore throwable from user's state machine. "
+ "Check if throwable " + compactedThrowable.throwableClassName()
+ " is present in the classpath.");
fut.completeExceptionally(new IgniteException(compactedThrowable.throwableMessage()));
}
} else if (th instanceof SMFullThrowable) {
fut.completeExceptionally(((SMFullThrowable) th).throwable());
}
}
private void scheduleRetry(Runnable runnable) {
executor.schedule(runnable, configuration.retryDelay().value(), TimeUnit.MILLISECONDS);
}
/**
* Checks if an error is recoverable, for example, {@link java.net.ConnectException}.
*
* @param t The throwable.
* @return {@code True} if this is a recoverable exception.
*/
private static boolean recoverable(Throwable t) {
if (t instanceof ExecutionException || t instanceof CompletionException) {
t = t.getCause();
}
return t instanceof TimeoutException || t instanceof IOException;
}
private Peer randomNode() {
return randomNode(null);
}
/**
* Returns a random peer. Tries 5 times finding a peer different from the excluded peer. If excluded peer is null, just returns a random
* peer.
*
* @param excludedPeer Excluded peer.
* @return Random peer.
*/
private Peer randomNode(@Nullable Peer excludedPeer) {
List<Peer> peers0 = peers;
// TODO https://issues.apache.org/jira/browse/IGNITE-19466
// assert peers0 != null && !peers0.isEmpty();
if (peers0 == null || peers0.isEmpty()) {
throw new IgniteInternalException(INTERNAL_ERR, "Peers are not ready [groupId=" + groupId + ']');
}
if (peers0.size() == 1) {
return peers0.get(0);
}
int lastPeerIndex = excludedPeer == null ? -1 : peers0.indexOf(excludedPeer);
ThreadLocalRandom random = current();
int newIdx = 0;
for (int retries = 0; retries < 5; retries++) {
newIdx = random.nextInt(peers0.size());
if (newIdx != lastPeerIndex) {
Peer peer = peers0.get(newIdx);
if (cluster.topologyService().getByConsistentId(peer.consistentId()) != null) {
break;
}
}
}
return peers0.get(newIdx);
}
/**
* Parse {@link Peer} from string representation of {@link PeerId}.
*
* @param peerId String representation of {@link PeerId}
* @return Peer
*/
private static @Nullable Peer parsePeer(@Nullable String peerId) {
PeerId id = PeerId.parsePeer(peerId);
return id == null ? null : new Peer(id.getConsistentId(), id.getIdx());
}
/**
* Parse list of {@link PeerId} from list with string representations.
*
* @param peers List of {@link PeerId} string representations.
* @return List of {@link PeerId}
*/
private static @Nullable List<Peer> parsePeerList(@Nullable Collection<String> peers) {
if (peers == null) {
return null;
}
List<Peer> res = new ArrayList<>(peers.size());
for (String peer : peers) {
res.add(parsePeer(peer));
}
return res;
}
private static String peerId(Peer peer) {
return PeerId.fromPeer(peer).toString();
}
private static List<String> peerIds(Collection<Peer> peers) {
return peers.stream().map(RaftGroupServiceImpl::peerId).collect(toList());
}
private CompletableFuture<ClusterNode> resolvePeer(Peer peer) {
ClusterNode node = cluster.topologyService().getByConsistentId(peer.consistentId());
if (node == null) {
return CompletableFuture.failedFuture(new ConnectException("Peer " + peer.consistentId() + " is unavailable"));
}
return CompletableFuture.completedFuture(node);
}
}