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 2021/12/21 01:54:05 UTC

[ratis] branch master updated: RATIS-1473.Implement takeSnapshot in Server (#567)

This is an automated email from the ASF dual-hosted git repository.

szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new cea19a6  RATIS-1473.Implement takeSnapshot in Server (#567)
cea19a6 is described below

commit cea19a648b93124354e3c57729b3888a65253a7e
Author: Yaolong Liu <ly...@163.com>
AuthorDate: Tue Dec 21 09:54:00 2021 +0800

    RATIS-1473.Implement takeSnapshot in Server (#567)
---
 .../org/apache/ratis/protocol/SnapshotRequest.java |  25 +++++
 .../apache/ratis/server/impl/RaftServerImpl.java   |  41 +++++++
 .../apache/ratis/server/impl/RaftServerProxy.java  |   4 +
 .../org/apache/ratis/server/impl/ServerState.java  |   4 +
 .../ratis/server/impl/SnapshotRequestHandler.java  | 123 +++++++++++++++++++++
 .../ratis/server/impl/StateMachineUpdater.java     |  11 ++
 .../ratis/server/impl/RaftServerTestUtil.java      |   9 ++
 .../server/simulation/SimulatedServerRpc.java      |   1 -
 .../ratis/statemachine/SnapshotManagementTest.java |  90 +++++++++++++++
 .../TestSnapshotManagementWithSimulatedRpc.java    |  25 +++++
 10 files changed, 332 insertions(+), 1 deletion(-)

diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotRequest.java
new file mode 100644
index 0000000..5d4c9aa
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotRequest.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.protocol;
+
+public final class SnapshotRequest extends RaftClientRequest {
+
+  public SnapshotRequest(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId,long callId, long timeoutMs) {
+    super(clientId, serverId, groupId, callId, readRequestType(), timeoutMs);
+  }
+}
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index eebcd15..6e2ac4d 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -59,6 +59,7 @@ import org.apache.ratis.server.raftlog.RaftLogIOException;
 import org.apache.ratis.server.storage.RaftStorage;
 import org.apache.ratis.server.storage.RaftStorageDirectory;
 import org.apache.ratis.server.util.ServerStringUtils;
+import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
@@ -178,6 +179,7 @@ class RaftServerImpl implements RaftServer.Division,
   private final AtomicBoolean startComplete;
 
   private final TransferLeadership transferLeadership;
+  private final SnapshotRequestHandler snapshotRequestHandler;
 
   RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy) throws IOException {
     final RaftPeerId id = proxy.getId();
@@ -218,6 +220,7 @@ class RaftServerImpl implements RaftServer.Division,
     });
 
     this.transferLeadership = new TransferLeadership(this);
+    this.snapshotRequestHandler = new SnapshotRequestHandler(this);
   }
 
   @Override
@@ -612,6 +615,13 @@ class RaftServerImpl implements RaftServer.Division,
         .build();
   }
 
+  RaftClientReply newSuccessReply(RaftClientRequest request, long logIndex) {
+    return newReplyBuilder(request)
+        .setSuccess()
+        .setLogIndex(logIndex)
+        .build();
+  }
+
   RaftClientReply newExceptionReply(RaftClientRequest request, RaftException exception) {
     return newReplyBuilder(request)
         .setException(exception)
@@ -964,6 +974,37 @@ class RaftServerImpl implements RaftServer.Division,
     }
   }
 
+  CompletableFuture<RaftClientReply> takeSnapshotAsync(SnapshotRequest request) throws IOException {
+    LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request);
+    assertLifeCycleState(LifeCycle.States.RUNNING);
+    assertGroup(request.getRequestorId(), request.getRaftGroupId());
+
+    //TODO(liuyaolong): make the min gap configurable, or get the gap value from shell command
+    long minGapValue = 5;
+    final Long lastSnapshotIndex = Optional.ofNullable(stateMachine.getLatestSnapshot())
+        .map(SnapshotInfo::getIndex)
+        .orElse(null);
+    if (lastSnapshotIndex != null && state.getLastAppliedIndex() - lastSnapshotIndex < minGapValue) {
+      return CompletableFuture.completedFuture(newSuccessReply(request, lastSnapshotIndex));
+    }
+
+    synchronized (this) {
+      long installSnapshot = inProgressInstallSnapshotRequest.get();
+      // check snapshot install/load
+      if (installSnapshot != 0) {
+        String msg = String.format("%s: Failed do snapshot as snapshot (%s) installation is in progress",
+            getMemberId(), installSnapshot);
+        LOG.warn(msg);
+        return CompletableFuture.completedFuture(newExceptionReply(request,new RaftException(msg)));
+      }
+      return snapshotRequestHandler.takingSnapshotAsync(request);
+    }
+  }
+
+  SnapshotRequestHandler getSnapshotRequestHandler() {
+    return snapshotRequestHandler;
+  }
+
   public RaftClientReply setConfiguration(SetConfigurationRequest request) throws IOException {
     return waitForReply(request, setConfigurationAsync(request));
   }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
index f03be97..f9dc2bb 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
@@ -530,6 +530,10 @@ class RaftServerProxy implements RaftServer {
         server -> server.getGroupInfo(request));
   }
 
+  public CompletableFuture<RaftClientReply> takeSnapshotAsync(SnapshotRequest request) {
+    return submitRequest(request.getRaftGroupId(), impl -> impl.takeSnapshotAsync(request));
+  }
+
   /**
    * Handle a raft configuration change request from client.
    */
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
index 38a37a1..2558ad4 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
@@ -403,6 +403,10 @@ class ServerState implements Closeable {
     return false;
   }
 
+  void notifyStateMachineUpdater() {
+    stateMachineUpdater.notifyUpdater();
+  }
+
   void reloadStateMachine(long lastIndexInSnapshot) {
     log.updateSnapshotIndex(lastIndexInSnapshot);
     stateMachineUpdater.reloadStateMachine();
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotRequestHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotRequestHandler.java
new file mode 100644
index 0000000..9e2f355
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotRequestHandler.java
@@ -0,0 +1,123 @@
+/*
+ * 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.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.SnapshotRequest;
+import org.apache.ratis.protocol.exceptions.TimeoutIOException;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.MemoizedSupplier;
+import org.apache.ratis.util.TimeDuration;
+import org.apache.ratis.util.TimeoutScheduler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+
+class SnapshotRequestHandler {
+  public static final Logger LOG = LoggerFactory.getLogger(TransferLeadership.class);
+
+  class PendingRequest {
+    private final SnapshotRequest request;
+    private final CompletableFuture<RaftClientReply> replyFuture = new CompletableFuture<>();
+    private final AtomicBoolean triggerTakingSnapshot = new AtomicBoolean(true);
+
+    PendingRequest(SnapshotRequest request) {
+      LOG.info("new PendingRequest " + request);
+      this.request = request;
+    }
+
+    CompletableFuture<RaftClientReply> getReplyFuture() {
+      return replyFuture;
+    }
+
+    boolean shouldTriggerTakingSnapshot() {
+      return triggerTakingSnapshot.getAndSet(false);
+    }
+
+    void complete(long index) {
+      LOG.info("{}: Successfully take snapshot at index {} for request {}", server.getMemberId(), index, request);
+      replyFuture.complete(server.newSuccessReply(request, index));
+    }
+
+    void timeout() {
+      replyFuture.completeExceptionally(new TimeoutIOException(
+          server.getMemberId() + ": Failed to take a snapshot within timeout " + request.getTimeoutMs() + "ms"));
+    }
+
+
+    @Override
+    public String toString() {
+      return request.toString();
+    }
+  }
+
+  static class PendingRequestReference {
+    private final AtomicReference<PendingRequest> ref = new AtomicReference<>();
+
+    Optional<PendingRequest> get() {
+      return Optional.ofNullable(ref.get());
+    }
+
+    Optional<PendingRequest> getAndSetNull() {
+      return Optional.ofNullable(ref.getAndSet(null));
+    }
+
+    PendingRequest getAndUpdate(Supplier<PendingRequest> supplier) {
+      return ref.getAndUpdate(p -> p != null? p: supplier.get());
+    }
+  }
+
+  private final RaftServerImpl server;
+  private final TimeoutScheduler scheduler = TimeoutScheduler.getInstance();
+  private final PendingRequestReference pending = new PendingRequestReference();
+
+  SnapshotRequestHandler(RaftServerImpl server) {
+    this.server = server;
+  }
+
+  CompletableFuture<RaftClientReply> takingSnapshotAsync(SnapshotRequest request) {
+    final MemoizedSupplier<PendingRequest> supplier = JavaUtils.memoize(() -> new PendingRequest(request));
+    final PendingRequest previous = pending.getAndUpdate(supplier);
+    if (previous != null) {
+      return previous.getReplyFuture();
+    }
+
+    server.getState().notifyStateMachineUpdater();
+    scheduler.onTimeout(TimeDuration.valueOf(request.getTimeoutMs(), TimeUnit.MILLISECONDS),
+        this::timeout, LOG, () -> "Timeout check failed for snapshot request: " + request);
+    return supplier.get().getReplyFuture();
+  }
+
+  boolean shouldTriggerTakingSnapshot() {
+    return pending.get().map(PendingRequest::shouldTriggerTakingSnapshot).orElse(false);
+  }
+
+  void completeTakingSnapshot(long index) {
+    pending.getAndSetNull().ifPresent(p -> p.complete(index));
+  }
+
+  void timeout() {
+    pending.getAndSetNull().ifPresent(PendingRequest::timeout);
+  }
+}
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
index e0b5348..efd9316 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
@@ -42,6 +42,7 @@ import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Consumer;
 import java.util.stream.LongStream;
@@ -80,6 +81,8 @@ class StateMachineUpdater implements Runnable {
   private final RaftLogIndex snapshotIndex;
   private final AtomicReference<Long> stopIndex = new AtomicReference<>();
   private volatile State state = State.RUNNING;
+  private final AtomicBoolean notified = new AtomicBoolean();
+
   private SnapshotRetentionPolicy snapshotRetentionPolicy;
   private StateMachineMetrics stateMachineMetrics = null;
 
@@ -158,6 +161,7 @@ class StateMachineUpdater implements Runnable {
 
   @SuppressFBWarnings("NN_NAKED_NOTIFY")
   synchronized void notifyUpdater() {
+    notified.set(true);
     notifyAll();
   }
 
@@ -202,6 +206,9 @@ class StateMachineUpdater implements Runnable {
     final long applied = getLastAppliedIndex();
     for(; applied >= raftLog.getLastCommittedIndex() && state == State.RUNNING && !shouldStop(); ) {
       wait();
+      if (notified.getAndSet(false)) {
+        return;
+      }
     }
   }
 
@@ -264,6 +271,7 @@ class StateMachineUpdater implements Runnable {
       Timer.Context takeSnapshotTimerContext = stateMachineMetrics.getTakeSnapshotTimer().time();
       i = stateMachine.takeSnapshot();
       takeSnapshotTimerContext.stop();
+      server.getSnapshotRequestHandler().completeTakingSnapshot(i);
 
       final long lastAppliedIndex = getLastAppliedIndex();
       if (i > lastAppliedIndex) {
@@ -300,6 +308,9 @@ class StateMachineUpdater implements Runnable {
   }
 
   private boolean shouldTakeSnapshot() {
+    if (state == State.RUNNING && server.getSnapshotRequestHandler().shouldTriggerTakingSnapshot()) {
+      return true;
+    }
     if (autoSnapshotThreshold == null) {
       return false;
     } else if (shouldStop()) {
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
index 74cc079..de0474a 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
@@ -20,10 +20,12 @@ package org.apache.ratis.server.impl;
 import org.apache.log4j.Level;
 import org.apache.ratis.RaftTestUtil;
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftGroupMemberId;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.SnapshotRequest;
 import org.apache.ratis.server.DataStreamMap;
 import org.apache.ratis.server.DataStreamServer;
 import org.apache.ratis.server.DivisionInfo;
@@ -41,9 +43,11 @@ import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
 import java.util.stream.Stream;
 
 public class RaftServerTestUtil {
@@ -176,4 +180,9 @@ public class RaftServerTestUtil {
   public static boolean isHighestPriority(RaftConfiguration config, RaftPeerId peerId) {
     return ((RaftConfigurationImpl)config).isHighestPriority(peerId);
   }
+
+  public static CompletableFuture<RaftClientReply> takeSnapshotAsync(RaftServer.Division leader, SnapshotRequest r)
+      throws IOException {
+    return ((RaftServerImpl)leader).takeSnapshotAsync(r);
+  }
 }
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
index 1e648dc..081253f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
@@ -17,7 +17,6 @@
  */
 package org.apache.ratis.server.simulation;
 
-import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
 import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
 import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto;
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java
new file mode 100644
index 0000000..47a19ba
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.BaseTest;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.SnapshotRequest;
+import org.apache.ratis.rpc.CallId;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.MiniRaftCluster;
+import org.apache.ratis.server.impl.RaftServerTestUtil;
+import org.apache.ratis.server.raftlog.RaftLog;
+import org.apache.ratis.util.Log4jUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+public abstract class SnapshotManagementTest<CLUSTER extends MiniRaftCluster>
+    extends BaseTest
+    implements MiniRaftCluster.Factory.Get<CLUSTER> {
+  {
+    Log4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG);
+    Log4jUtils.setLogLevel(RaftLog.LOG, Level.INFO);
+    Log4jUtils.setLogLevel(RaftClient.LOG, Level.INFO);
+  }
+
+  static final Logger LOG = LoggerFactory.getLogger(SnapshotManagementTest.class);
+
+  @Before
+  public void setup() {
+    final RaftProperties p = getProperties();
+    p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SimpleStateMachine4Testing.class, StateMachine.class);
+    RaftServerConfigKeys.Snapshot.setAutoTriggerEnabled(p, false);
+  }
+
+  @Test
+  public void testTakeSnapshot() throws Exception {
+    runWithNewCluster(1, this::runTestTakeSnapshot);
+  }
+
+  void runTestTakeSnapshot(CLUSTER cluster) throws Exception {
+    final RaftClientReply snapshotReply;
+    final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster);
+    final RaftPeerId leaderId = leader.getId();
+    try (final RaftClient client = cluster.createClient(leaderId)) {
+      //todo(yaolong liu) : make 5 to be a configurable value
+      for (int i = 0; i < 5; i++) {
+        RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i));
+        Assert.assertTrue(reply.isSuccess());
+      }
+      final SnapshotRequest r = new SnapshotRequest(client.getId(), leaderId, cluster.getGroupId(),
+          CallId.getAndIncrement(), 3000);
+      snapshotReply = RaftServerTestUtil.takeSnapshotAsync(leader, r).join();
+    }
+
+    Assert.assertTrue(snapshotReply.isSuccess());
+    final long snapshotIndex = snapshotReply.getLogIndex();
+    LOG.info("snapshotIndex = {}", snapshotIndex);
+
+    final File snapshotFile = SimpleStateMachine4Testing.get(leader)
+        .getStateMachineStorage().getSnapshotFile(leader.getInfo().getCurrentTerm(), snapshotIndex);
+    Assert.assertTrue(snapshotFile.exists());
+  }
+}
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestSnapshotManagementWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestSnapshotManagementWithSimulatedRpc.java
new file mode 100644
index 0000000..ea8729c
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestSnapshotManagementWithSimulatedRpc.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.statemachine.SnapshotManagementTest;
+
+public class TestSnapshotManagementWithSimulatedRpc
+    extends SnapshotManagementTest<MiniRaftClusterWithSimulatedRpc>
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+}