sanpwc commented on code in PR #4543:
URL: https://github.com/apache/ignite-3/pull/4543#discussion_r1797170114
##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java:
##########
@@ -687,36 +690,50 @@ private void handleErrorResponse(
case EBUSY:
case EAGAIN:
- scheduleRetry(() -> sendWithRetry(peer, requestFactory,
stopTime, fut, retryCount + 1));
+ scheduleRetry(peer, requestFactory, stopTime, fut, retryCount);
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 ChangePeersAndLearnersAsyncRequest) {
- sendWithRetry(randomNode(peer), requestFactory,
stopTime, fut, retryCount + 1);
- } else {
- sendWithRetry(peer, requestFactory, stopTime, fut,
retryCount + 1);
- }
- });
+ case ENOENT: {
+ Peer newTargetPeer;
+
+ // 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 ChangePeersAndLearnersAsyncRequest) {
+ newTargetPeer = randomNode(peer);
+ } else {
+ newTargetPeer = peer;
+ }
+
+ scheduleRetry(newTargetPeer, requestFactory, stopTime, fut,
retryCount);
break;
+ }
+ case EHOSTDOWN:
+ case ESHUTDOWN:
+ case ENODESHUTDOWN:
case EPERM:
// TODO: IGNITE-15706
case UNKNOWN:
- case EINTERNAL:
+ case EINTERNAL: {
+ Peer newTargetPeer;
+
if (resp.leaderId() == null) {
- scheduleRetry(() -> sendWithRetry(randomNode(peer),
requestFactory, stopTime, fut, retryCount + 1));
+ newTargetPeer = randomNode(peer);
Review Comment:
Generally that doesn't look right. First of all, because of the fact that we
will hide the fact that all nodes were down with TimeoutException. Besides that
it's rather not a good idea to retry the request on same stopped node. Yep,
formally it's possible that it'll be restarted soon enough, but in real life we
will barely face it.
Basically we already have a retryContext: stopTime, fut, retryCount. I'd
literally incapsulate that into new RetryContext class along with adding
stoppedPeers set (or excludeFromRetiresPeers, naming might be different). If
all possible nodes are stopped - complete the future with new corresponding
exception. Of course that also means that it's required to handle it properly.
Initially handling might be the same as TimeoutException one with TODO's for
further enhancements.
##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java:
##########
@@ -608,16 +607,20 @@ private <R extends NetworkMessage> void sendWithRetry(
err == null ? null : err.getMessage());
}
- if (err != null) {
- handleThrowable(err, peer, request,
requestFactory, stopTime, fut, retryCount);
- } else if (resp instanceof ErrorResponse) {
- handleErrorResponse((ErrorResponse) resp, peer,
request, requestFactory, stopTime, fut, retryCount);
- } else if (resp instanceof SMErrorResponse) {
- handleSmErrorResponse((SMErrorResponse) resp, fut);
- } else {
- leader = peer; // The OK response was received
from a leader.
-
- fut.complete((R) resp);
+ try {
+ if (err != null) {
+ handleThrowable(err, peer, request,
requestFactory, stopTime, fut, retryCount);
+ } else if (resp instanceof ErrorResponse) {
+ handleErrorResponse((ErrorResponse) resp,
peer, request, requestFactory, stopTime, fut, retryCount);
+ } else if (resp instanceof SMErrorResponse) {
+ handleSmErrorResponse((SMErrorResponse) resp,
fut);
+ } else {
+ leader = peer; // The OK response was received
from a leader.
+
+ fut.complete((R) resp);
+ }
+ } catch (Throwable e) {
Review Comment:
What kind of exceptions do you expect here?
Seems that we already have (exceptions (!errorResponses)) handling logic
incapsulated inside
```
if (err != null) {
handleThrowable(err, peer, request,
requestFactory, stopTime, fut, retryCount);
}
```
##########
modules/raft/src/test/java/org/apache/ignite/internal/raft/RaftGroupServiceTest.java:
##########
@@ -623,6 +628,28 @@ public void testReadIndexWithMessageSendTimeout() {
assertThat(fut, willThrowFast(TimeoutException.class));
}
+ @ParameterizedTest
+ @EnumSource(names = {"ESHUTDOWN", "EHOSTDOWN", "ENODESHUTDOWN", "EPERM",
"UNKNOWN", "EINTERNAL"})
+ public void testRetryOnErrorWithUpdateLeader(RaftError error) {
Review Comment:
Seems that test that asserts expected exception in case of all down nodes is
missing.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]