You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by dr...@apache.org on 2022/10/17 12:09:51 UTC

[ratis] branch master updated: RATIS-1722. Fix the suppressed findbugs warnings in SimpleStateMachineStorage. (#760)

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

dragonyliu 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 77a990651 RATIS-1722. Fix the suppressed findbugs warnings in SimpleStateMachineStorage. (#760)
77a990651 is described below

commit 77a990651d16d0857665281d777983085e180575
Author: Tsz-Wo Nicholas Sze <sz...@apache.org>
AuthorDate: Mon Oct 17 20:09:46 2022 +0800

    RATIS-1722. Fix the suppressed findbugs warnings in SimpleStateMachineStorage. (#760)
---
 .../examples/arithmetic/TestArithmeticLogDump.java |   2 +-
 .../impl/SimpleStateMachineStorage.java            | 136 +++++++++++----------
 .../ratis/InstallSnapshotFromLeaderTests.java      |   7 +-
 .../ratis/InstallSnapshotNotificationTests.java    |   8 +-
 .../java/org/apache/ratis/LogAppenderTests.java    |   2 +-
 .../org/apache/ratis/MessageStreamApiTests.java    |   2 +-
 .../org/apache/ratis/RaftAsyncExceptionTests.java  |   2 +-
 .../test/java/org/apache/ratis/RaftAsyncTests.java |   2 +-
 .../apache/ratis/RequestLimitAsyncBaseTest.java    |   2 +-
 .../java/org/apache/ratis/WatchRequestTests.java   |   2 +-
 .../server/impl/PreAppendLeaderStepDownTest.java   |   2 +-
 .../impl/RaftStateMachineExceptionTests.java       |   2 +-
 .../server/impl/StateMachineShutdownTests.java     |   2 +-
 .../ratis/statemachine/RaftSnapshotBaseTest.java   |   1 +
 .../ratis/statemachine/SnapshotManagementTest.java |   2 +-
 .../{ => impl}/SimpleStateMachine4Testing.java     |  19 ++-
 .../ratis/TestRaftServerNoLeaderTimeout.java       |   2 +-
 .../ratis/TestRaftServerSlownessDetection.java     |   2 +-
 .../apache/ratis/grpc/TestLogAppenderWithGrpc.java |   2 +-
 .../apache/ratis/grpc/TestRaftServerWithGrpc.java  |   2 +-
 .../org/apache/ratis/grpc/TestRaftWithGrpc.java    |   2 +-
 .../apache/ratis/grpc/TestRetryCacheWithGrpc.java  |   2 +-
 .../ratis/retry/TestExceptionDependentRetry.java   |   2 +-
 .../apache/ratis/server/ServerRestartTests.java    |   2 +-
 .../server/raftlog/memory/MemoryRaftLogTest.java   |   3 +-
 .../raftlog/segmented/TestCacheEviction.java       |   2 +-
 .../raftlog/segmented/TestSegmentedRaftLog.java    |   2 +-
 .../cli/sh/ElectionCommandIntegrationTest.java     |   4 +-
 .../shell/cli/sh/GroupCommandIntegrationTest.java  |   4 +-
 .../shell/cli/sh/PeerCommandIntegrationTest.java   |   4 +-
 .../cli/sh/SnapshotCommandIntegrationTest.java     |   4 +-
 .../ratis/statemachine/TestStateMachine.java       |   1 +
 32 files changed, 121 insertions(+), 112 deletions(-)

diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java
index b88267997..73ec91cac 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java
@@ -30,7 +30,7 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.raftlog.segmented.LogSegmentPath;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.tools.ParseRatisLog;
 import org.apache.ratis.util.Log4jUtils;
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java
index 9bcafdde1..1be9cb281 100644
--- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java
@@ -17,11 +17,11 @@
  */
 package org.apache.ratis.statemachine.impl;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import org.apache.ratis.io.MD5Hash;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.FileInfo;
 import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.SnapshotRetentionPolicy;
 import org.apache.ratis.statemachine.StateMachineStorage;
 import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -38,7 +38,10 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Comparator;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -56,15 +59,13 @@ public class SimpleStateMachineStorage implements StateMachineStorage {
   public static final Pattern SNAPSHOT_REGEX =
       Pattern.compile(SNAPSHOT_FILE_PREFIX + "\\.(\\d+)_(\\d+)");
 
-  private RaftStorage raftStorage;
-  private File smDir = null;
+  private volatile File stateMachineDir = null;
 
   private volatile SingleFileSnapshotInfo currentSnapshot = null;
 
   @Override
-  public void init(RaftStorage rStorage) throws IOException {
-    this.raftStorage = rStorage;
-    this.smDir = raftStorage.getStorageDir().getStateMachineDir();
+  public void init(RaftStorage storage) throws IOException {
+    this.stateMachineDir = storage.getStorageDir().getStateMachineDir();
     loadLatestSnapshot();
   }
 
@@ -73,35 +74,49 @@ public class SimpleStateMachineStorage implements StateMachineStorage {
     // TODO
   }
 
-  @Override
-  @SuppressFBWarnings("NP_NULL_ON_SOME_PATH")
-  public void cleanupOldSnapshots(SnapshotRetentionPolicy snapshotRetentionPolicy) throws IOException {
-    if (snapshotRetentionPolicy != null && snapshotRetentionPolicy.getNumSnapshotsRetained() > 0) {
-
-      List<SingleFileSnapshotInfo> allSnapshotFiles = new ArrayList<>();
-      try (DirectoryStream<Path> stream =
-               Files.newDirectoryStream(smDir.toPath())) {
-        for (Path path : stream) {
-          Matcher matcher = SNAPSHOT_REGEX.matcher(path.getFileName().toString());
+  static List<SingleFileSnapshotInfo> getSingleFileSnapshotInfos(Path dir) throws IOException {
+    final List<SingleFileSnapshotInfo> infos = new ArrayList<>();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir)) {
+      for (Path path : stream) {
+        final Path filename = path.getFileName();
+        if (filename != null) {
+          final Matcher matcher = SNAPSHOT_REGEX.matcher(filename.toString());
           if (matcher.matches()) {
-            final long endIndex = Long.parseLong(matcher.group(2));
             final long term = Long.parseLong(matcher.group(1));
-            final FileInfo fileInfo = new FileInfo(path, null); //We don't need FileDigest here.
-            allSnapshotFiles.add(new SingleFileSnapshotInfo(fileInfo, term, endIndex));
+            final long index = Long.parseLong(matcher.group(2));
+            final FileInfo fileInfo = new FileInfo(path, null); //No FileDigest here.
+            infos.add(new SingleFileSnapshotInfo(fileInfo, term, index));
           }
         }
       }
+    }
+    return infos;
+  }
 
-      if (allSnapshotFiles.size() > snapshotRetentionPolicy.getNumSnapshotsRetained()) {
-        allSnapshotFiles.sort(new SnapshotFileComparator());
-        List<File> snapshotFilesToBeCleaned = allSnapshotFiles.subList(
-            snapshotRetentionPolicy.getNumSnapshotsRetained(), allSnapshotFiles.size()).stream()
-            .map(singleFileSnapshotInfo -> singleFileSnapshotInfo.getFile().getPath().toFile())
-            .collect(Collectors.toList());
-        for (File snapshotFile : snapshotFilesToBeCleaned) {
-          LOG.info("Deleting old snapshot at {}", snapshotFile.getAbsolutePath());
-          FileUtils.deleteFileQuietly(snapshotFile);
-        }
+  @Override
+  public void cleanupOldSnapshots(SnapshotRetentionPolicy snapshotRetentionPolicy) throws IOException {
+    if (stateMachineDir == null) {
+      return;
+    }
+
+    final int numSnapshotsRetained = Optional.ofNullable(snapshotRetentionPolicy)
+        .map(SnapshotRetentionPolicy::getNumSnapshotsRetained)
+        .orElse(SnapshotRetentionPolicy.DEFAULT_ALL_SNAPSHOTS_RETAINED);
+    if (numSnapshotsRetained <= 0) {
+      return;
+    }
+
+    final List<SingleFileSnapshotInfo> allSnapshotFiles = getSingleFileSnapshotInfos(stateMachineDir.toPath());
+
+    if (allSnapshotFiles.size() > snapshotRetentionPolicy.getNumSnapshotsRetained()) {
+      allSnapshotFiles.sort(Comparator.comparing(SnapshotInfo::getIndex).reversed());
+      List<File> snapshotFilesToBeCleaned = allSnapshotFiles.subList(
+              snapshotRetentionPolicy.getNumSnapshotsRetained(), allSnapshotFiles.size()).stream()
+          .map(singleFileSnapshotInfo -> singleFileSnapshotInfo.getFile().getPath().toFile())
+          .collect(Collectors.toList());
+      for (File snapshotFile : snapshotFilesToBeCleaned) {
+        LOG.info("Deleting old snapshot at {}", snapshotFile.getAbsolutePath());
+        FileUtils.deleteFileQuietly(snapshotFile);
       }
     }
   }
@@ -128,40 +143,46 @@ public class SimpleStateMachineStorage implements StateMachineStorage {
   }
 
   public File getSnapshotFile(long term, long endIndex) {
-    return new File(smDir, getSnapshotFileName(term, endIndex));
+    final File dir = Objects.requireNonNull(stateMachineDir, "stateMachineDir == null");
+    return new File(dir, getSnapshotFileName(term, endIndex));
   }
 
   protected File getTmpSnapshotFile(long term, long endIndex) {
-    return new File(smDir, getTmpSnapshotFileName(term, endIndex));
+    final File dir = Objects.requireNonNull(stateMachineDir, "stateMachineDir == null");
+    return new File(dir, getTmpSnapshotFileName(term, endIndex));
   }
 
   protected File getCorruptSnapshotFile(long term, long endIndex) {
-    return new File(smDir, getCorruptSnapshotFileName(term, endIndex));
+    final File dir = Objects.requireNonNull(stateMachineDir, "stateMachineDir == null");
+    return new File(dir, getCorruptSnapshotFileName(term, endIndex));
   }
 
-  @SuppressFBWarnings("NP_NULL_ON_SOME_PATH")
-  public SingleFileSnapshotInfo findLatestSnapshot() throws IOException {
-    SingleFileSnapshotInfo latest = null;
-    try (DirectoryStream<Path> stream =
-             Files.newDirectoryStream(smDir.toPath())) {
-      for (Path path : stream) {
-        Matcher matcher = SNAPSHOT_REGEX.matcher(path.getFileName().toString());
-        if (matcher.matches()) {
-          final long endIndex = Long.parseLong(matcher.group(2));
-          if (latest == null || endIndex > latest.getIndex()) {
-            final long term = Long.parseLong(matcher.group(1));
-            MD5Hash fileDigest = MD5FileUtil.readStoredMd5ForFile(path.toFile());
-            final FileInfo fileInfo = new FileInfo(path, fileDigest);
-            latest = new SingleFileSnapshotInfo(fileInfo, term, endIndex);
-          }
-        }
+  static SingleFileSnapshotInfo findLatestSnapshot(Path dir) throws IOException {
+    final Iterator<SingleFileSnapshotInfo> i = getSingleFileSnapshotInfos(dir).iterator();
+    if (!i.hasNext()) {
+      return null;
+    }
+
+    SingleFileSnapshotInfo latest = i.next();
+    for(; i.hasNext(); ) {
+      final SingleFileSnapshotInfo info = i.next();
+      if (info.getIndex() > latest.getIndex()) {
+        latest = info;
       }
     }
-    return latest;
+
+    // read md5
+    final Path path = latest.getFile().getPath();
+    final MD5Hash md5 = MD5FileUtil.readStoredMd5ForFile(path.toFile());
+    final FileInfo info = new FileInfo(path, md5);
+    return new SingleFileSnapshotInfo(info, latest.getTerm(), latest.getIndex());
   }
 
   public void loadLatestSnapshot() throws IOException {
-    this.currentSnapshot = findLatestSnapshot();
+    if (stateMachineDir == null) {
+      return;
+    }
+    this.currentSnapshot = findLatestSnapshot(stateMachineDir.toPath());
   }
 
   public static String getSnapshotFileName(long term, long endIndex) {
@@ -174,18 +195,7 @@ public class SimpleStateMachineStorage implements StateMachineStorage {
   }
 
   @VisibleForTesting
-  public File getSmDir() {
-    return smDir;
-  }
-}
-
-/**
- * Compare snapshot files based on transaction indexes.
- */
-@SuppressFBWarnings("SE_COMPARATOR_SHOULD_BE_SERIALIZABLE")
-class SnapshotFileComparator implements Comparator<SingleFileSnapshotInfo> {
-  @Override
-  public int compare(SingleFileSnapshotInfo file1, SingleFileSnapshotInfo file2) {
-    return (int) (file2.getIndex() - file1.getIndex());
+  File getStateMachineDir() {
+    return stateMachineDir;
   }
 }
diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java
index bc4de2ff4..1cc0d3001 100644
--- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -30,13 +30,12 @@ import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.raftlog.RaftLog;
 import org.apache.ratis.server.storage.FileInfo;
 import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.impl.FileListSnapshotInfo;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.LifeCycle;
-import org.apache.ratis.util.MD5FileUtil;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -120,7 +119,7 @@ public abstract class InstallSnapshotFromLeaderTests<CLUSTER extends MiniRaftClu
             // contains two snapshot files
             // sm/snapshot/1.bin
             // sm/snapshot/sub/2.bin
-            snapshotRoot = new File(getSMdir(), "snapshot");
+            snapshotRoot = new File(getStateMachineDir(), "snapshot");
             file1 = new File(snapshotRoot, "1.bin");
             file2 = new File(new File(snapshotRoot, "sub"), "2.bin");
         }
diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java
index 215e8408f..cbaf6bff8 100644
--- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java
@@ -32,7 +32,7 @@ import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.raftlog.RaftLog;
 import org.apache.ratis.server.raftlog.segmented.LogSegmentPath;
 import org.apache.ratis.statemachine.RaftSnapshotBaseTest;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
@@ -109,7 +109,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC
       Supplier<TermIndex> supplier = () -> {
         try {
           Path leaderSnapshotFile = leaderSnapshotInfo.getFile().getPath();
-          File followerSnapshotFilePath = new File(getSMdir(),
+          final File followerSnapshotFilePath = new File(getStateMachineDir(),
               leaderSnapshotFile.getFileName().toString());
           // simulate the real situation such as snapshot transmission delay
           Thread.sleep(1000);
@@ -148,7 +148,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC
             }
           } else {
             LOG.info("Receive the notification to clean up snapshot as follower for {}, result: {}", peer, result);
-            File followerSnapshotFile = new File(getSMdir(), leaderSnapshotFile.getName());
+            final File followerSnapshotFile = new File(getStateMachineDir(), leaderSnapshotFile.getName());
             if (followerSnapshotFile.exists()) {
               FileUtils.deleteFile(followerSnapshotFile);
               LOG.info("follower snapshot {} deleted", followerSnapshotFile);
@@ -481,7 +481,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC
       File leaderSnapshotFile = leaderSnapshotInfo.getFiles().get(0).getPath().toFile();
       SimpleStateMachine4Testing followerStateMachine =
           (SimpleStateMachine4Testing) cluster.getFollowers().get(0).getStateMachine();
-      File followerSnapshotFile = new File(followerStateMachine.getStateMachineStorage().getSmDir(),
+      final File followerSnapshotFile = new File(followerStateMachine.getStateMachineDir(),
           leaderSnapshotFile.getName());
       Assert.assertEquals(numNotifyInstallSnapshotFinished.get(), 2);
       Assert.assertTrue(leaderSnapshotFile.exists());
diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java
index da0c25f9c..ab4b891b3 100644
--- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java
@@ -36,7 +36,7 @@ import org.apache.ratis.server.impl.MiniRaftCluster;
 import org.apache.ratis.server.metrics.RaftServerMetricsImpl;
 import org.apache.ratis.server.raftlog.LogProtoUtils;
 import org.apache.ratis.server.raftlog.RaftLog;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Log4jUtils;
diff --git a/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java b/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java
index f48d8e175..a0612708c 100644
--- a/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java
@@ -27,7 +27,7 @@ import org.apache.ratis.protocol.Message;
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.impl.MiniRaftCluster;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.Log4jUtils;
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java
index 187cd0a01..8c723b87f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java
@@ -28,7 +28,7 @@ import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.server.impl.MiniRaftCluster;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.Log4jUtils;
 import org.junit.Assert;
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java
index 691ca90f9..d03327eb9 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java
@@ -42,7 +42,7 @@ import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
 import org.apache.ratis.server.impl.MiniRaftCluster;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
diff --git a/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java
index 36272daa3..095ac74ac 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java
@@ -30,7 +30,7 @@ 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.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.util.Log4jUtils;
 import org.junit.Assert;
 import org.junit.Test;
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 266716b09..a6658dafb 100644
--- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
@@ -34,7 +34,7 @@ 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.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.Log4jUtils;
 import org.apache.ratis.util.ProtoUtils;
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java
index aa7f636c8..93af35658 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java
@@ -29,7 +29,7 @@ import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.protocol.exceptions.StateMachineException;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.raftlog.RaftLog;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.JavaUtils;
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
index 91ce6c1eb..687a4d35a 100644
--- 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
@@ -29,7 +29,7 @@ import org.apache.ratis.protocol.exceptions.StateMachineException;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RetryCache;
 import org.apache.ratis.server.raftlog.RaftLog;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.JavaUtils;
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java
index 9692eefcb..28f8e6ace 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java
@@ -25,7 +25,7 @@ import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.RaftServer;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.junit.Assert;
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
index 1e4094342..da8eb165e 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
@@ -43,6 +43,7 @@ import org.apache.ratis.server.metrics.RaftServerMetricsImpl;
 import org.apache.ratis.server.raftlog.RaftLog;
 import org.apache.ratis.server.raftlog.segmented.LogSegmentPath;
 import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.JavaUtils;
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
index c821f36c4..39c5caf52 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java
@@ -29,8 +29,8 @@ 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.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.util.Log4jUtils;
 import org.junit.Assert;
 import org.junit.Before;
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java
similarity index 97%
rename from ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
rename to ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java
index cf715585e..141c221ba 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ratis.statemachine;
+package org.apache.ratis.statemachine.impl;
 
 import org.apache.ratis.RaftTestUtil.SimpleMessage;
 import org.apache.ratis.conf.RaftProperties;
@@ -37,9 +37,8 @@ import org.apache.ratis.server.raftlog.RaftLog;
 import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogInputStream;
 import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogOutputStream;
 import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.statemachine.impl.BaseStateMachine;
-import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
-import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.Daemon;
 import org.apache.ratis.util.JavaUtils;
@@ -66,8 +65,8 @@ import java.util.concurrent.TimeUnit;
 /**
  * A {@link StateMachine} implementation example that simply stores all the log
  * entries in a list. Mainly used for test.
- *
- * For snapshot it simply merges all the log segments together.
+ * <p>
+ * For snapshots, it simply merges all the log segments together.
  */
 public class SimpleStateMachine4Testing extends BaseStateMachine {
   private static final int SNAPSHOT_THRESHOLD = 100;
@@ -214,7 +213,7 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
     getLifeCycle().startAndTransition(() -> {
       super.initialize(server, groupId, raftStorage);
       storage.init(raftStorage);
-      loadSnapshot(storage.findLatestSnapshot());
+      loadSnapshot(storage.getLatestSnapshot());
 
       if (properties.getBoolean(
           RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_KEY,
@@ -233,7 +232,7 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
   @Override
   public synchronized void reinitialize() throws IOException {
     LOG.info("Reinitializing " + this);
-    loadSnapshot(storage.findLatestSnapshot());
+    loadSnapshot(storage.findLatestSnapshot(getStateMachineDir().toPath()));
     if (getLifeCycleState() == LifeCycle.State.PAUSED) {
       getLifeCycle().transition(LifeCycle.State.STARTING);
       getLifeCycle().transition(LifeCycle.State.RUNNING);
@@ -439,7 +438,7 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
     return notifiedAsLeader;
   }
 
-  protected File getSMdir() {
-    return storage.getSmDir();
+  public File getStateMachineDir() {
+    return storage.getStateMachineDir();
   }
 }
diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java
index fc1ecb40d..8492db507 100644
--- a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java
+++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java
@@ -25,7 +25,7 @@ import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.MiniRaftCluster;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
 import org.apache.ratis.proto.RaftProtos;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.Log4jUtils;
 import org.apache.ratis.util.TimeDuration;
diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
index db2a8df46..224ac646c 100644
--- a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
+++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
@@ -27,7 +27,7 @@ import org.apache.ratis.server.impl.MiniRaftCluster;
 import org.apache.ratis.server.metrics.RaftServerMetricsImpl;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
 import org.apache.ratis.proto.RaftProtos;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.Log4jUtils;
 import org.apache.ratis.util.TimeDuration;
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java
index c3ad38486..0d90da5a0 100644
--- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java
@@ -29,7 +29,7 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.leader.FollowerInfo;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Log4jUtils;
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java
index dd36b0032..cdc076162 100644
--- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java
@@ -50,7 +50,7 @@ import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.server.metrics.RaftServerMetricsImpl;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.Log4jUtils;
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
index 86f8f376a..bc0061f5f 100644
--- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
@@ -26,7 +26,7 @@ import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.server.raftlog.LogEntryHeader;
 import org.apache.ratis.server.raftlog.RaftLog;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.TimeDuration;
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
index 5456ebbd3..400e6e5a6 100644
--- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
@@ -28,7 +28,7 @@ import org.apache.ratis.protocol.exceptions.ResourceUnavailableException;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RetryCacheTestUtil;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.junit.Test;
 
diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java
index a219175d3..7eba6a82b 100644
--- a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java
+++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java
@@ -30,7 +30,7 @@ import org.apache.ratis.protocol.exceptions.RaftRetryFailureException;
 import org.apache.ratis.protocol.exceptions.TimeoutIOException;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.TimeDuration;
 import org.junit.Assert;
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java
index 2c0e18167..f1e0652e0 100644
--- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java
+++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java
@@ -38,7 +38,7 @@ import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogFormat;
 import org.apache.ratis.server.RaftServerConfigKeys.Log;
 import org.apache.ratis.server.raftlog.segmented.TestSegmentedRaftLog;
 import org.apache.ratis.server.raftlog.segmented.LogSegmentPath;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.JavaUtils;
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java
index 086c10dea..fec64f0a1 100644
--- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java
+++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java
@@ -30,10 +30,9 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftGroupMemberId;
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.raftlog.LogEntryHeader;
 import org.apache.ratis.server.raftlog.RaftLog;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.Log4jUtils;
 import org.junit.Test;
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java
index 87dd2ef37..3acc17164 100644
--- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java
+++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java
@@ -35,7 +35,7 @@ import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogCache.LogSegmen
 import org.apache.ratis.server.raftlog.segmented.TestSegmentedRaftLog.SegmentRange;
 import org.apache.ratis.server.storage.RaftStorage;
 import org.apache.ratis.server.storage.RaftStorageTestUtils;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.SizeInBytes;
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java
index f8b9c960d..e66cb7153 100644
--- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java
+++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java
@@ -40,7 +40,7 @@ import org.apache.ratis.server.raftlog.LogProtoUtils;
 import org.apache.ratis.server.raftlog.RaftLog;
 import org.apache.ratis.server.storage.RaftStorage;
 import org.apache.ratis.server.storage.RaftStorageTestUtils;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.impl.BaseStateMachine;
 import org.apache.ratis.util.LifeCycle;
diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java
index d68300b26..e88becb57 100644
--- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java
+++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -25,7 +25,7 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.MiniRaftCluster;
 import org.apache.ratis.server.raftlog.RaftLog;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Log4jUtils;
diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java
index 515cbcaeb..93d4e7bbd 100644
--- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java
+++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -25,7 +25,7 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.MiniRaftCluster;
 import org.apache.ratis.server.raftlog.RaftLog;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.Log4jUtils;
 import org.apache.ratis.util.SizeInBytes;
diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java
index 99d95553b..ef0a3a7db 100644
--- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java
+++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -27,7 +27,7 @@ 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.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.thirdparty.com.google.common.collect.ObjectArrays;
 import org.apache.ratis.util.JavaUtils;
diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java
index fbb804a43..8d9af8c3c 100644
--- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java
+++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -27,7 +27,7 @@ import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.MiniRaftCluster;
 import org.apache.ratis.server.raftlog.RaftLog;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.Log4jUtils;
 import org.apache.ratis.util.SizeInBytes;
diff --git a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
index 039fe1cb5..d4d307178 100644
--- a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
+++ b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
@@ -34,6 +34,7 @@ import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
+import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing;
 import org.apache.ratis.util.Log4jUtils;
 import org.junit.*;