This is an automated email from the ASF dual-hosted git repository.
msingh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-ratis.git
The following commit(s) were added to refs/heads/master by this push:
new 43ab53c RATIS-575. TestWatchRequestWithGrpc#testWatchRequestTimeout
fails with NotReplicatedException. Contributed by Tsz Wo Nicholas Sze.
43ab53c is described below
commit 43ab53c3940b6ac745188f9c38d904635d37cb80
Author: Mukul Kumar Singh <[email protected]>
AuthorDate: Fri Jun 28 13:13:10 2019 +0530
RATIS-575. TestWatchRequestWithGrpc#testWatchRequestTimeout fails with
NotReplicatedException. Contributed by Tsz Wo Nicholas Sze.
---
.../java/org/apache/ratis/WatchRequestTests.java | 25 ++++++++++++++++------
1 file changed, 18 insertions(+), 7 deletions(-)
diff --git a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
index 86b6e4c..ebc5352 100644
--- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
@@ -33,6 +33,7 @@ import org.apache.ratis.util.LogUtils;
import org.apache.ratis.util.ProtoUtils;
import org.apache.ratis.util.TimeDuration;
import org.apache.ratis.util.function.CheckedConsumer;
+import org.apache.ratis.util.function.CheckedSupplier;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -42,6 +43,7 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;
@@ -398,17 +400,26 @@ public abstract class WatchRequestTests<CLUSTER extends
MiniRaftCluster>
final WatchReplies watchReplies = watches.get(i).get(GET_TIMEOUT_SECOND,
TimeUnit.SECONDS);
Assert.assertEquals(logIndex, watchReplies.logIndex);
- final RaftClientReply watchAllReply = watchReplies.getAll();
- assertNotReplicatedException(logIndex, ReplicationLevel.ALL,
watchAllReply);
+ assertNotReplicatedException(logIndex, ReplicationLevel.ALL,
watchReplies::getAll);
- final RaftClientReply watchAllCommittedReply =
watchReplies.getAllCommitted();
- assertNotReplicatedException(logIndex, ReplicationLevel.ALL_COMMITTED,
watchAllCommittedReply);
+ assertNotReplicatedException(logIndex, ReplicationLevel.ALL_COMMITTED,
watchReplies::getAllCommitted);
+ }
+ }
+
+ static void assertNotReplicatedException(long logIndex, ReplicationLevel
replication,
+ CheckedSupplier<RaftClientReply, Exception> replySupplier) throws
Exception {
+ try {
+ replySupplier.get();
+ Assert.fail();
+ } catch (ExecutionException e) {
+ final Throwable cause = e.getCause();
+ assertNotReplicatedException(logIndex, replication, cause);
}
}
- static void assertNotReplicatedException(long logIndex, ReplicationLevel
replication, RaftClientReply reply) {
- Assert.assertFalse(reply.isSuccess());
- final NotReplicatedException nre = reply.getNotReplicatedException();
+ static void assertNotReplicatedException(long logIndex, ReplicationLevel
replication, Throwable t) {
+ Assert.assertSame(NotReplicatedException.class, t.getClass());
+ final NotReplicatedException nre = (NotReplicatedException) t;
Assert.assertNotNull(nre);
Assert.assertEquals(logIndex, nre.getLogIndex());
Assert.assertEquals(replication, nre.getRequiredReplication());