This is an automated email from the ASF dual-hosted git repository.
ptupitsyn 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 b246612777f IGNITE-27529 Fix circular exception chain in
ClientFutureUtils.doWithRetryAsync (#7384)
b246612777f is described below
commit b246612777fd4a6bbe6289ecc5bfffcc13db6de9
Author: Pavel Tupitsyn <[email protected]>
AuthorDate: Mon Jan 12 11:59:00 2026 +0200
IGNITE-27529 Fix circular exception chain in
ClientFutureUtils.doWithRetryAsync (#7384)
* Skip duplicates when adding suppressed exceptions in
`ClientFutureUtils.doWithRetryAsync`
* Skip obsolete address holders in `ReliableChannel` (root cause of
duplicate exceptions during retries)
---
.../ignite/internal/client/ClientFutureUtils.java | 28 ++++++++++++++-
.../ignite/internal/client/ReliableChannel.java | 2 +-
.../internal/client/ClientFutureUtilsTest.java | 40 ++++++++++++++++++++++
3 files changed, 68 insertions(+), 2 deletions(-)
diff --git
a/modules/client/src/main/java/org/apache/ignite/internal/client/ClientFutureUtils.java
b/modules/client/src/main/java/org/apache/ignite/internal/client/ClientFutureUtils.java
index ca13ca608dd..6960cf781b1 100644
---
a/modules/client/src/main/java/org/apache/ignite/internal/client/ClientFutureUtils.java
+++
b/modules/client/src/main/java/org/apache/ignite/internal/client/ClientFutureUtils.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.client;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.concurrent.CompletableFuture;
import java.util.function.Predicate;
import java.util.function.Supplier;
@@ -74,8 +75,15 @@ class ClientFutureUtils {
} else {
resErr = ctx.errors.get(0);
+ HashSet<Throwable> dejaVu = new HashSet<>();
+ existingCauseOrSuppressed(resErr, dejaVu); // Seed
dejaVu.
+
for (int i = 1; i < ctx.errors.size(); i++) {
- resErr.addSuppressed(ctx.errors.get(i));
+ Throwable e = ctx.errors.get(i);
+
+ if (!existingCauseOrSuppressed(e, dejaVu)) {
+ resErr.addSuppressed(e);
+ }
}
}
}
@@ -91,6 +99,24 @@ class ClientFutureUtils {
});
}
+ private static boolean existingCauseOrSuppressed(Throwable t,
HashSet<Throwable> dejaVu) {
+ if (t == null) {
+ return false;
+ }
+
+ if (!dejaVu.add(t)) {
+ return true;
+ }
+
+ for (Throwable sup : t.getSuppressed()) {
+ if (existingCauseOrSuppressed(sup, dejaVu)) {
+ return true;
+ }
+ }
+
+ return existingCauseOrSuppressed(t.getCause(), dejaVu);
+ }
+
static class RetryContext {
int attempt;
diff --git
a/modules/client/src/main/java/org/apache/ignite/internal/client/ReliableChannel.java
b/modules/client/src/main/java/org/apache/ignite/internal/client/ReliableChannel.java
index 3d54918e751..168b8f819cf 100644
---
a/modules/client/src/main/java/org/apache/ignite/internal/client/ReliableChannel.java
+++
b/modules/client/src/main/java/org/apache/ignite/internal/client/ReliableChannel.java
@@ -410,7 +410,7 @@ public final class ReliableChannel implements AutoCloseable
{
if (preferredNodeName != null) {
ClientChannelHolder holder =
nodeChannelsByName.get(preferredNodeName);
- if (holder != null) {
+ if (holder != null && !holder.close) {
return holder.getOrCreateChannelAsync().thenCompose(ch -> {
if (ch != null) {
return completedFuture(ch);
diff --git
a/modules/client/src/test/java/org/apache/ignite/internal/client/ClientFutureUtilsTest.java
b/modules/client/src/test/java/org/apache/ignite/internal/client/ClientFutureUtilsTest.java
index 6920b10fd04..cfba1e939d0 100644
---
a/modules/client/src/test/java/org/apache/ignite/internal/client/ClientFutureUtilsTest.java
+++
b/modules/client/src/test/java/org/apache/ignite/internal/client/ClientFutureUtilsTest.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.client;
import static
org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@@ -108,4 +109,43 @@ public class ClientFutureUtilsTest {
var ex = assertThrows(CompletionException.class, fut::join);
assertEquals("fail1", ex.getCause().getMessage());
}
+
+ @Test
+ public void
testDoWithRetryAsyncPreventsDuplicatesAndSelfReferenceInSuppressedExceptions() {
+ var counter = new AtomicInteger();
+
+ var ex1 = new Exception("1");
+ var ex2 = new Exception("2");
+
+ var fut = ClientFutureUtils.doWithRetryAsync(
+ () -> {
+ switch (counter.get()) {
+ case 0: // Self.
+ return CompletableFuture.failedFuture(ex1);
+
+ case 1: // Other.
+ return CompletableFuture.failedFuture(ex2);
+
+ case 2: // Self wrapped.
+ return CompletableFuture.failedFuture(new
Exception(ex1));
+
+ case 3: // Other wrapped.
+ return CompletableFuture.failedFuture(new
Exception(ex2));
+
+ default:
+ return CompletableFuture.failedFuture(new
Exception("Other"));
+ }
+ },
+ ctx -> counter.incrementAndGet() < 4
+ );
+
+ var completionEx = assertThrows(CompletionException.class, fut::join);
+ var ex = (Exception) completionEx.getCause();
+
+ assertEquals(ex1, ex, "Expected the first exception to be the main
one.");
+
+ Throwable[] suppressed = ex.getSuppressed();
+ assertEquals(1, suppressed.length, "Should not have duplicate
suppressed exceptions.");
+ assertEquals(ex2, unwrapCause(suppressed[0]));
+ }
}