You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sz...@apache.org on 2018/09/10 20:09:29 UTC

incubator-ratis git commit: RATIS-303. TestRaftStateMachineException is failing with NullPointerException. Contributed by Shashikant Banerjee

Repository: incubator-ratis
Updated Branches:
  refs/heads/master ef48512da -> 89b1a1cd9


RATIS-303. TestRaftStateMachineException is failing with NullPointerException.  Contributed by Shashikant Banerjee


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/89b1a1cd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/89b1a1cd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/89b1a1cd

Branch: refs/heads/master
Commit: 89b1a1cd9b12623f031845881e3df069dcafe249
Parents: ef48512
Author: Tsz Wo Nicholas Sze <sz...@apache.org>
Authored: Mon Sep 10 13:08:28 2018 -0700
Committer: Tsz Wo Nicholas Sze <sz...@apache.org>
Committed: Mon Sep 10 13:08:28 2018 -0700

----------------------------------------------------------------------
 .../TestRaftStateMachineException.java          | 170 -----------------
 .../TestRaftStateMachineExceptionWithGrpc.java  |  26 +++
 ...tRaftStateMachineExceptionWithHadoopRpc.java |  25 +++
 .../TestRaftStateMachineExceptionWithNetty.java |  25 +++
 .../impl/RaftStateMachineExceptionTests.java    | 188 +++++++++++++++++++
 ...ftStateMachineExceptionWithSimulatedRpc.java |  25 +++
 .../SimpleStateMachine4Testing.java             |   2 +-
 7 files changed, 290 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/89b1a1cd/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java b/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
deleted file mode 100644
index a339b68..0000000
--- a/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ratis.statemachine;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.RaftTestUtil.SimpleMessage;
-import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.client.RaftClientRpc;
-import org.apache.ratis.examples.ParameterizedBaseTest;
-import org.apache.ratis.protocol.*;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.server.impl.RaftServerTestUtil;
-import org.apache.ratis.server.impl.RetryCache;
-import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.util.LogUtils;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-
-import static org.junit.Assert.fail;
-
-public class TestRaftStateMachineException extends ParameterizedBaseTest {
-  static {
-    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    LogUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  protected static boolean failPreAppend = false;
-
-  protected static class StateMachineWithException extends SimpleStateMachine4Testing {
-    @Override
-    public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
-      CompletableFuture<Message> future = new CompletableFuture<>();
-      future.completeExceptionally(new StateMachineException("Fake Exception"));
-      return future;
-    }
-
-    @Override
-    public TransactionContext preAppendTransaction(TransactionContext trx)
-        throws IOException {
-      if (failPreAppend) {
-        throw new IOException("Fake Exception in preAppend");
-      } else {
-        return trx;
-      }
-    }
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() throws IOException {
-    return getMiniRaftClusters(StateMachineWithException.class, 3);
-  }
-
-  @Parameterized.Parameter
-  public MiniRaftCluster cluster;
-
-  @Test
-  public void testHandleStateMachineException() throws Exception {
-    setAndStart(cluster);
-
-    final RaftPeerId leaderId = cluster.getLeader().getId();
-
-    try(final RaftClient client = cluster.createClient(leaderId)) {
-      client.send(new SimpleMessage("m"));
-      fail("Exception expected");
-    } catch (StateMachineException e) {
-      e.printStackTrace();
-      Assert.assertTrue(e.getCause().getMessage().contains("Fake Exception"));
-    }
-  }
-
-  @Test
-  public void testRetryOnStateMachineException() throws Exception {
-    setAndStart(cluster);
-
-    final RaftPeerId leaderId = cluster.getLeaderAndSendFirstMessage(true).getId();
-    long oldLastApplied = cluster.getLeader().getState().getLastAppliedIndex();
-
-    final RaftClient client = cluster.createClient(leaderId);
-    final RaftClientRpc rpc = client.getClientRpc();
-    final long callId = 999;
-    final long seqNum = 111;
-    RaftClientRequest r = cluster.newRaftClientRequest(client.getId(), leaderId,
-        callId, seqNum, new SimpleMessage("message"));
-    RaftClientReply reply = rpc.sendRequest(r);
-    Assert.assertFalse(reply.isSuccess());
-    Assert.assertNotNull(reply.getStateMachineException());
-
-    // retry with the same callId
-    for (int i = 0; i < 5; i++) {
-      reply = rpc.sendRequest(r);
-      Assert.assertEquals(client.getId(), reply.getClientId());
-      Assert.assertEquals(callId, reply.getCallId());
-      Assert.assertFalse(reply.isSuccess());
-      Assert.assertNotNull(reply.getStateMachineException());
-    }
-
-    long leaderApplied = cluster.getLeader().getState().getLastAppliedIndex();
-    // make sure retry cache has the entry
-    for (RaftServerImpl server : cluster.iterateServerImpls()) {
-      LOG.info("check server " + server.getId());
-      if (server.getState().getLastAppliedIndex() < leaderApplied) {
-        Thread.sleep(1000);
-      }
-      Assert.assertNotNull(
-          RaftServerTestUtil.getRetryEntry(server, client.getId(), callId));
-      Assert.assertEquals(oldLastApplied + 1,
-          server.getState().getLastAppliedIndex());
-    }
-
-    client.close();
-  }
-
-  @Test
-  public void testRetryOnExceptionDuringReplication() throws Exception {
-    setAndStart(cluster);
-    final RaftPeerId leaderId = cluster.getLeaderAndSendFirstMessage(true).getId();
-
-    // turn on the preAppend failure switch
-    failPreAppend = true;
-    final RaftClient client = cluster.createClient(leaderId);
-    final RaftClientRpc rpc = client.getClientRpc();
-    final long callId = 999;
-    final long seqNum = 111;
-    RaftClientRequest r = cluster.newRaftClientRequest(client.getId(), leaderId,
-        callId, seqNum, new SimpleMessage("message"));
-    RaftClientReply reply = rpc.sendRequest(r);
-    Objects.requireNonNull(reply.getStateMachineException());
-
-    RetryCache.CacheEntry oldEntry = RaftServerTestUtil.getRetryEntry(
-        cluster.getLeader(), client.getId(), callId);
-    Assert.assertNotNull(oldEntry);
-    Assert.assertTrue(RaftServerTestUtil.isRetryCacheEntryFailed(oldEntry));
-
-    // retry
-    reply = rpc.sendRequest(r);
-    Objects.requireNonNull(reply.getStateMachineException());
-
-    RetryCache.CacheEntry currentEntry = RaftServerTestUtil.getRetryEntry(
-        cluster.getLeader(), client.getId(), callId);
-    Assert.assertNotNull(currentEntry);
-    Assert.assertTrue(RaftServerTestUtil.isRetryCacheEntryFailed(currentEntry));
-    Assert.assertNotEquals(oldEntry, currentEntry);
-
-    failPreAppend = false;
-    client.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/89b1a1cd/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java
new file mode 100644
index 0000000..ca36738
--- /dev/null
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.grpc;
+
+import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
+
+public class TestRaftStateMachineExceptionWithGrpc
+    extends RaftStateMachineExceptionTests<MiniRaftClusterWithGRpc>
+    implements MiniRaftClusterWithGRpc.FactoryGet {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/89b1a1cd/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftStateMachineExceptionWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftStateMachineExceptionWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftStateMachineExceptionWithHadoopRpc.java
new file mode 100644
index 0000000..6a8fd9d
--- /dev/null
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftStateMachineExceptionWithHadoopRpc.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.hadooprpc;
+
+import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
+
+public class TestRaftStateMachineExceptionWithHadoopRpc extends
+    RaftStateMachineExceptionTests<MiniRaftClusterWithHadoopRpc>
+    implements MiniRaftClusterWithHadoopRpc.Factory.Get {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/89b1a1cd/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java
new file mode 100644
index 0000000..34fddc9
--- /dev/null
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.netty;
+
+import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
+
+public class TestRaftStateMachineExceptionWithNetty
+    extends RaftStateMachineExceptionTests<MiniRaftClusterWithNetty>
+    implements MiniRaftClusterWithNetty.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/89b1a1cd/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java
new file mode 100644
index 0000000..ec635d0
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java
@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.server.impl;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.BaseTest;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.client.RaftClientRpc;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.*;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.util.LogUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.fail;
+
+public abstract class RaftStateMachineExceptionTests<CLUSTER extends MiniRaftCluster>
+    extends BaseTest implements MiniRaftCluster.Factory.Get<CLUSTER> {
+  {
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  protected static boolean failPreAppend = false;
+
+  protected static class StateMachineWithException extends
+      SimpleStateMachine4Testing {
+
+    @Override
+    public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
+      CompletableFuture<Message> future = new CompletableFuture<>();
+      future.completeExceptionally(new StateMachineException("Fake Exception"));
+      return future;
+    }
+
+    @Override
+    public TransactionContext preAppendTransaction(TransactionContext trx)
+        throws IOException {
+      if (failPreAppend) {
+        throw new IOException("Fake Exception in preAppend");
+      } else {
+        return trx;
+      }
+    }
+  }
+
+  @Test
+  public void testHandleStateMachineException() throws Exception {
+    final RaftProperties prop = getProperties();
+    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        StateMachineWithException.class, StateMachine.class);
+    final MiniRaftCluster cluster = newCluster(3);
+    cluster.start();
+
+    RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId();
+
+    try(final RaftClient client = cluster.createClient(leaderId)) {
+      client.send(new RaftTestUtil.SimpleMessage("m"));
+      fail("Exception expected");
+    } catch (StateMachineException e) {
+      e.printStackTrace();
+      Assert.assertTrue(e.getCause().getMessage().contains("Fake Exception"));
+    }
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testRetryOnStateMachineException() throws Exception {
+    final RaftProperties prop = getProperties();
+    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        StateMachineWithException.class, StateMachine.class);
+    final MiniRaftCluster cluster = newCluster(3);
+    cluster.start();
+
+    RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId();
+
+    cluster.getLeaderAndSendFirstMessage(true);
+    long oldLastApplied = cluster.getLeader().getState().getLastAppliedIndex();
+
+    final RaftClient client = cluster.createClient(leaderId);
+    final RaftClientRpc rpc = client.getClientRpc();
+    final long callId = 999;
+    final long seqNum = 111;
+    RaftClientRequest r = cluster.newRaftClientRequest(client.getId(), leaderId,
+        callId, seqNum, new RaftTestUtil.SimpleMessage("message"));
+    RaftClientReply reply = rpc.sendRequest(r);
+    Assert.assertFalse(reply.isSuccess());
+    Assert.assertNotNull(reply.getStateMachineException());
+
+    // retry with the same callId
+    for (int i = 0; i < 5; i++) {
+      reply = rpc.sendRequest(r);
+      Assert.assertEquals(client.getId(), reply.getClientId());
+      Assert.assertEquals(callId, reply.getCallId());
+      Assert.assertFalse(reply.isSuccess());
+      Assert.assertNotNull(reply.getStateMachineException());
+    }
+
+    long leaderApplied = cluster.getLeader().getState().getLastAppliedIndex();
+    // make sure retry cache has the entry
+    for (RaftServerImpl server : cluster.iterateServerImpls()) {
+      LOG.info("check server " + server.getId());
+      if (server.getState().getLastAppliedIndex() < leaderApplied) {
+        Thread.sleep(1000);
+      }
+      Assert.assertNotNull(
+          RaftServerTestUtil.getRetryEntry(server, client.getId(), callId));
+      Assert.assertEquals(oldLastApplied + 1,
+          server.getState().getLastAppliedIndex());
+    }
+
+    client.close();
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testRetryOnExceptionDuringReplication() throws Exception {
+    final RaftProperties prop = getProperties();
+    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        StateMachineWithException.class, StateMachine.class);
+    final MiniRaftCluster cluster = newCluster(3);
+    cluster.start();
+    RaftTestUtil.waitForLeader(cluster);
+    RaftServerImpl leader = cluster.getLeader();
+    RaftPeerId leaderId = leader.getId();
+    cluster.getLeaderAndSendFirstMessage(true);
+    // turn on the preAppend failure switch
+    failPreAppend = true;
+    final RaftClient client = cluster.createClient(leaderId);
+    final RaftClientRpc rpc = client.getClientRpc();
+    final long callId = 999;
+    final long seqNum = 111;
+    RaftClientRequest r = cluster.newRaftClientRequest(client.getId(), leaderId,
+        callId, seqNum, new RaftTestUtil.SimpleMessage("message"));
+    RaftClientReply reply = rpc.sendRequest(r);
+    Objects.requireNonNull(reply.getStateMachineException());
+
+    RetryCache.CacheEntry oldEntry = RaftServerTestUtil.getRetryEntry(
+        leader, client.getId(), callId);
+    Assert.assertNotNull(oldEntry);
+    Assert.assertTrue(RaftServerTestUtil.isRetryCacheEntryFailed(oldEntry));
+
+    // At this point of time the old leader would have stepped down. wait for
+    // leader election to complete
+    RaftTestUtil.waitForLeader(cluster);
+    leader = cluster.getLeader();
+    leaderId = leader.getId();
+    // retry
+    r = cluster.newRaftClientRequest(client.getId(), leaderId,
+        callId, seqNum, new RaftTestUtil.SimpleMessage("message"));
+    reply = rpc.sendRequest(r);
+    Objects.requireNonNull(reply.getStateMachineException());
+
+    RetryCache.CacheEntry currentEntry = RaftServerTestUtil.getRetryEntry(
+        leader, client.getId(), callId);
+    Assert.assertNotNull(currentEntry);
+    Assert.assertTrue(RaftServerTestUtil.isRetryCacheEntryFailed(currentEntry));
+    Assert.assertNotEquals(oldEntry, currentEntry);
+    failPreAppend = false;
+    client.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/89b1a1cd/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java
new file mode 100644
index 0000000..d8ca86f
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.server.simulation;
+
+import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
+
+public class TestRaftStateMachineExceptionWithSimulatedRpc extends
+    RaftStateMachineExceptionTests<MiniRaftClusterWithSimulatedRpc>
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/89b1a1cd/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
index 5554da2..b6d6d3f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
@@ -88,7 +88,7 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
   private RoleInfoProto slownessInfo = null;
   private RoleInfoProto leaderElectionTimeoutInfo = null;
 
-  SimpleStateMachine4Testing() {
+  public SimpleStateMachine4Testing() {
     checkpointer = new Daemon(() -> {
       while (running) {
           if (list.get(list.size() - 1).getIndex() - endIndexLastCkpt >=