This is an automated email from the ASF dual-hosted git repository.
sanpwc pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new 2fa9121 IGNITE-16751 TimeoutException handling within
RaftGroupServiceImpl#sendWithRetry fixed
2fa9121 is described below
commit 2fa9121d2a90abf4eb48f31de3cebccf403c7056
Author: Alexander Lapin <[email protected]>
AuthorDate: Wed Mar 30 10:58:22 2022 +0300
IGNITE-16751 TimeoutException handling within
RaftGroupServiceImpl#sendWithRetry fixed
---
.../java/org/apache/ignite/internal/raft/ItLozaTest.java | 2 +-
.../apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java | 8 +++++++-
.../org/apache/ignite/raft/jraft/core/RaftGroupServiceTest.java | 2 +-
3 files changed, 9 insertions(+), 3 deletions(-)
diff --git
a/modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLozaTest.java
b/modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLozaTest.java
index 2c80c9c..891f4c2 100644
---
a/modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLozaTest.java
+++
b/modules/raft/src/integrationTest/java/org/apache/ignite/internal/raft/ItLozaTest.java
@@ -113,7 +113,7 @@ public class ItLozaTest {
when(service.messagingService()).thenReturn(messagingServiceMock);
- CompletableFuture<NetworkMessage> exception =
CompletableFuture.failedFuture(new Exception(new IOException()));
+ CompletableFuture<NetworkMessage> exception =
CompletableFuture.failedFuture(new IOException());
loza = new Loza(service, dataPath);
diff --git
a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java
b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java
index 37495fe..5f84a35 100644
---
a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java
+++
b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/rpc/impl/RaftGroupServiceImpl.java
@@ -43,6 +43,8 @@ import java.util.Collection;
import java.util.Collections;
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.TimeUnit;
import java.util.concurrent.TimeoutException;
@@ -613,7 +615,11 @@ public class RaftGroupServiceImpl implements
RaftGroupService {
* @return {@code True} if this is a recoverable exception.
*/
private boolean recoverable(Throwable t) {
- return t instanceof TimeoutException || t.getCause() instanceof
IOException;
+ if (t instanceof ExecutionException || t instanceof
CompletionException) {
+ t = t.getCause();
+ }
+
+ return t instanceof TimeoutException || t instanceof IOException;
}
/**
diff --git
a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/RaftGroupServiceTest.java
b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/RaftGroupServiceTest.java
index 9202138..2650c19 100644
---
a/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/RaftGroupServiceTest.java
+++
b/modules/raft/src/test/java/org/apache/ignite/raft/jraft/core/RaftGroupServiceTest.java
@@ -793,7 +793,7 @@ public class RaftGroupServiceTest {
NetworkAddress target = invocation.getArgument(0);
if (peer != null && target.equals(peer.address()))
- return failedFuture(new IgniteInternalException(new
ConnectException()));
+ return failedFuture(new ConnectException());
if (delay) {
return CompletableFuture.supplyAsync(() -> {