Skip to content

Commit a7e4de0

Browse files
committed
IGNITE-28123 Under node restarts, partition assignment lookup may complete exceptionally while resolving primary replicas.
In ClientPrimaryReplicaTracker.primaryReplicasAsyncInternal, the exceptional path was not terminated for non-TableNotFound errors, and execution could continue into result construction with partitions == null, causing a secondary failure (NullPointerException from unboxing).
1 parent 251d567 commit a7e4de0

2 files changed

Lines changed: 24 additions & 1 deletion

File tree

modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientPrimaryReplicaTracker.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -197,7 +197,9 @@ private CompletableFuture<PrimaryReplicasResult> primaryReplicasAsyncInternal(in
197197
throw new CompletionException(cause);
198198
}
199199

200-
assert false : "Unexpected error: " + err;
200+
throw err instanceof CompletionException
201+
? (CompletionException) err
202+
: new CompletionException(err);
201203
}
202204

203205
PrimaryReplicasResult res = primaryReplicasNoWait(tableId, maxStartTime0, timestamp, true);

modules/client-handler/src/test/java/org/apache/ignite/client/handler/ClientPrimaryReplicaTrackerTest.java

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,11 +20,13 @@
2020
import static org.junit.jupiter.api.Assertions.assertEquals;
2121
import static org.junit.jupiter.api.Assertions.assertNull;
2222
import static org.junit.jupiter.api.Assertions.assertTrue;
23+
import static org.junit.jupiter.api.Assertions.fail;
2324
import static org.mockito.Mockito.mock;
2425
import static org.mockito.Mockito.when;
2526

2627
import java.util.List;
2728
import java.util.concurrent.CompletableFuture;
29+
import java.util.concurrent.CompletionException;
2830
import java.util.concurrent.atomic.AtomicLong;
2931
import org.apache.ignite.client.handler.ClientPrimaryReplicaTracker.PrimaryReplicasResult;
3032
import org.apache.ignite.internal.TestHybridClock;
@@ -134,6 +136,25 @@ public void testFailedInitialFutureIsRetried() {
134136
assertEquals("s2", replicas.nodeNames().get(1));
135137
}
136138

139+
@Test
140+
public void testFailedPrimaryReplicaRequestPropagatesOriginalError() {
141+
driver.returnError(true);
142+
tracker.start();
143+
144+
try {
145+
tracker.primaryReplicasAsync(TABLE_ID, null).join();
146+
fail("Expected completion to fail");
147+
} catch (CompletionException e) {
148+
Throwable cause = e.getCause();
149+
while (cause instanceof CompletionException && cause.getCause() != null) {
150+
cause = cause.getCause();
151+
}
152+
153+
assertTrue(cause instanceof RuntimeException);
154+
assertEquals("FakePlacementDriver expected error", cause.getMessage());
155+
}
156+
}
157+
137158
@Test
138159
public void testOldEventsAreIgnoredByLeaseStartTime() {
139160
tracker.start();

0 commit comments

Comments
 (0)