You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by ru...@apache.org on 2020/12/18 05:16:32 UTC

[incubator-ratis] branch master updated: RATIS-1250. Move RaftStorage classes to ratis-server-api. (#362)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new b4b18ca  RATIS-1250. Move RaftStorage classes to ratis-server-api. (#362)
b4b18ca is described below

commit b4b18ca6632b6399fc6d5372b8340d34f1ad66b7
Author: Tsz-Wo Nicholas Sze <sz...@apache.org>
AuthorDate: Fri Dec 18 13:16:22 2020 +0800

    RATIS-1250. Move RaftStorage classes to ratis-server-api. (#362)
---
 .../apache/ratis/util/AtomicFileOutputStream.java  |  8 ++-
 .../org/apache/ratis/server/storage/FileInfo.java  |  2 +-
 .../apache/ratis/server/storage/RaftStorage.java   | 49 +++++------------
 .../ratis/server/storage/RaftStorageDirectory.java | 53 +++++++++++++++++++
 .../apache/ratis/server/impl/RaftServerImpl.java   |  2 +-
 .../org/apache/ratis/server/impl/ServerState.java  |  4 +-
 .../server/raftlog/segmented/SegmentedRaftLog.java |  4 +-
 .../ratis/server/storage/FileChunkReader.java      |  6 ++-
 ...irectory.java => RaftStorageDirectoryImpl.java} | 61 +++++-----------------
 .../{RaftStorage.java => RaftStorageImpl.java}     | 24 ++++-----
 .../ratis/server/storage/SnapshotManager.java      |  6 ++-
 .../ratis/server/storage/RaftStorageTestUtils.java |  2 +-
 .../ratis/server/storage/TestRaftStorage.java      | 30 ++++++-----
 13 files changed, 130 insertions(+), 121 deletions(-)

diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
index 72608a6..b2381ba 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
@@ -43,16 +43,20 @@ import java.util.concurrent.atomic.AtomicBoolean;
  * before the temporary file is moved.
  */
 public class AtomicFileOutputStream extends FilterOutputStream {
+  static final Logger LOG = LoggerFactory.getLogger(AtomicFileOutputStream.class);
+
   public static final String TMP_EXTENSION = ".tmp";
 
-  public static final Logger LOG = LoggerFactory.getLogger(AtomicFileOutputStream.class);
+  public static File getTemporaryFile(File outFile) {
+    return new File(outFile.getParentFile(), outFile.getName() + TMP_EXTENSION);
+  }
 
   private final File outFile;
   private final File tmpFile;
   private final AtomicBoolean isClosed = new AtomicBoolean();
 
   public AtomicFileOutputStream(File outFile) throws FileNotFoundException {
-    this(outFile, new File(outFile.getParentFile(), outFile.getName() + TMP_EXTENSION));
+    this(outFile, getTemporaryFile(outFile));
   }
 
   public AtomicFileOutputStream(File outFile, File tmpFile) throws FileNotFoundException {
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java b/ratis-server-api/src/main/java/org/apache/ratis/server/storage/FileInfo.java
similarity index 99%
copy from ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java
copy to ratis-server-api/src/main/java/org/apache/ratis/server/storage/FileInfo.java
index 9673596..e972958 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java
+++ b/ratis-server-api/src/main/java/org/apache/ratis/server/storage/FileInfo.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
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java b/ratis-server-api/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
similarity index 51%
rename from ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java
rename to ratis-server-api/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
index 9673596..d8f4fec 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java
+++ b/ratis-server-api/src/main/java/org/apache/ratis/server/storage/RaftStorage.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
@@ -17,43 +17,22 @@
  */
 package org.apache.ratis.server.storage;
 
-import java.nio.file.Path;
+import org.apache.ratis.server.RaftServerConfigKeys.Log.CorruptionPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.ratis.io.MD5Hash;
+import java.io.Closeable;
 
-/**
- * Metadata about a file.
- *
- * The objects of this class are immutable.
- */
-public class FileInfo {
-  private final Path path;
-  private final MD5Hash fileDigest;
-  private final long fileSize;
-
-  public FileInfo(Path path, MD5Hash fileDigest) {
-    this.path = path;
-    this.fileDigest = fileDigest;
-    this.fileSize = path.toFile().length();
-  }
-
-  @Override
-  public String toString() {
-    return path.toString();
-  }
+/** The storage of a raft server. */
+public interface RaftStorage extends Closeable {
+  Logger LOG = LoggerFactory.getLogger(RaftStorage.class);
 
-  /** @return the path of the file. */
-  public Path getPath() {
-    return path;
-  }
+  /** @return the storage directory. */
+  RaftStorageDirectory getStorageDir();
 
-  /** @return the MD5 file digest of the file. */
-  public MD5Hash getFileDigest() {
-    return fileDigest;
-  }
+  /** @return the metadata file. */
+  RaftStorageMetadataFile getMetadataFile();
 
-  /** @return the size of the file. */
-  public long getFileSize() {
-    return fileSize;
-  }
+  /** @return the corruption policy for raft log. */
+  CorruptionPolicy getLogCorruptionPolicy();
 }
diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java b/ratis-server-api/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java
new file mode 100644
index 0000000..bad3929
--- /dev/null
+++ b/ratis-server-api/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.storage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+/** The directory of a {@link RaftStorage}. */
+public interface RaftStorageDirectory {
+  Logger LOG = LoggerFactory.getLogger(RaftStorageDirectory.class);
+
+  String CURRENT_DIR_NAME = "current";
+  String STATE_MACHINE_DIR_NAME = "sm"; // directory containing state machine snapshots
+  String TMP_DIR_NAME = "tmp";
+
+  /** @return the root directory of this storage */
+  File getRoot();
+
+  /** @return the current directory. */
+  default File getCurrentDir() {
+    return new File(getRoot(), CURRENT_DIR_NAME);
+  }
+
+  /** @return the state machine directory. */
+  default File getStateMachineDir() {
+    return new File(getRoot(), STATE_MACHINE_DIR_NAME);
+  }
+
+  /** @return the temporary directory. */
+  default File getTmpDir() {
+    return new File(getRoot(), TMP_DIR_NAME);
+  }
+
+  /** Is this storage healthy? */
+  boolean isHealthy();
+}
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 c1d2839..8939f4f 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
@@ -500,7 +500,7 @@ class RaftServerImpl implements RaftServer.Division,
 
   GroupInfoReply getGroupInfo(GroupInfoRequest request) {
     return new GroupInfoReply(request, getCommitInfos(),
-        getGroup(), getRoleInfoProto(), state.getStorage().getStorageDir().hasMetaFile());
+        getGroup(), getRoleInfoProto(), state.getStorage().getStorageDir().isHealthy());
   }
 
   RoleInfoProto getRoleInfoProto() {
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 806300e..e6b4674 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
@@ -67,7 +67,7 @@ class ServerState implements Closeable {
   /** The thread that applies committed log entries to the state machine */
   private final StateMachineUpdater stateMachineUpdater;
   /** local storage for log and snapshot */
-  private final RaftStorage storage;
+  private final RaftStorageImpl storage;
   private final SnapshotManager snapshotManager;
   private volatile Timestamp lastNoLeaderTime;
   private final TimeDuration noLeaderTimeout;
@@ -106,7 +106,7 @@ class ServerState implements Closeable {
     // use full uuid string to create a subdirectory
     final File dir = chooseStorageDir(RaftServerConfigKeys.storageDir(prop),
         group.getGroupId().getUuid().toString());
-    storage = new RaftStorage(dir, RaftServerConfigKeys.Log.corruptionPolicy(prop));
+    storage = new RaftStorageImpl(dir, RaftServerConfigKeys.Log.corruptionPolicy(prop));
     snapshotManager = new SnapshotManager(storage, id);
 
     initStatemachine(stateMachine, group.getGroupId());
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java
index 56bd0a1..b8e1f9a 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java
@@ -457,12 +457,12 @@ public class SegmentedRaftLog extends RaftLog {
 
   @Override
   public void writeMetadata(RaftStorageMetadata metadata) throws IOException {
-    storage.getMetaFile().persist(metadata);
+    storage.getMetadataFile().persist(metadata);
   }
 
   @Override
   public RaftStorageMetadata loadMetadata() throws IOException {
-    return storage.getMetaFile().getMetadata();
+    return storage.getMetadataFile().getMetadata();
   }
 
   @Override
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java
index 32b062f..a9288ef 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java
@@ -26,6 +26,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.nio.file.Path;
+import java.util.Optional;
 
 /** Read {@link FileChunkProto}s from a file. */
 public class FileChunkReader implements Closeable {
@@ -46,7 +47,10 @@ public class FileChunkReader implements Closeable {
    */
   public FileChunkReader(FileInfo info, RaftStorageDirectory directory) throws IOException {
     this.info = info;
-    this.relativePath = directory.relativizeToRoot(info.getPath());
+    this.relativePath = Optional.of(info.getPath())
+        .filter(Path::isAbsolute)
+        .map(p -> directory.getRoot().toPath().relativize(p))
+        .orElse(info.getPath());
     final File f = info.getPath().toFile();
     this.in = new FileInputStream(f);
   }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java
similarity index 82%
rename from ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java
rename to ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java
index 0f8e1df..5e69f7d 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java
@@ -19,8 +19,6 @@ package org.apache.ratis.server.storage;
 
 import org.apache.ratis.util.AtomicFileOutputStream;
 import org.apache.ratis.util.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
@@ -32,18 +30,14 @@ import java.nio.charset.StandardCharsets;
 import java.nio.file.DirectoryStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.*;
+import java.util.Objects;
 
 import static java.nio.file.Files.newDirectoryStream;
 
-public class RaftStorageDirectory {
-  static final Logger LOG = LoggerFactory.getLogger(RaftStorageDirectory.class);
+class RaftStorageDirectoryImpl implements RaftStorageDirectory {
 
-  static final String STORAGE_DIR_CURRENT = "current";
-  static final String STORAGE_FILE_LOCK = "in_use.lock";
-  static final String META_FILE_NAME = "raft-meta";
-  static final String STATE_MACHINE = "sm"; // directory containing state machine snapshots
-  static final String TEMP = "tmp";
+  private static final String IN_USE_LOCK_NAME = "in_use.lock";
+  private static final String META_FILE_NAME = "raft-meta";
   private static final String CONF_EXTENSION = ".conf";
 
   enum StorageState {
@@ -59,15 +53,12 @@ public class RaftStorageDirectory {
    * Constructor
    * @param dir directory corresponding to the storage
    */
-  RaftStorageDirectory(File dir) {
+  RaftStorageDirectoryImpl(File dir) {
     this.root = dir;
     this.lock = null;
   }
 
-  /**
-   * Get root directory of this storage
-   */
-  //TODO
+  @Override
   public File getRoot() {
     return root;
   }
@@ -96,46 +87,19 @@ public class RaftStorageDirectory {
     FileUtils.createDirectories(dir);
   }
 
-  /**
-   * Directory {@code current} contains latest files defining
-   * the file system meta-data.
-   *
-   * @return the directory path
-   */
-  public File getCurrentDir() {
-    return new File(root, STORAGE_DIR_CURRENT);
-  }
 
   File getMetaFile() {
     return new File(getCurrentDir(), META_FILE_NAME);
   }
 
   File getMetaTmpFile() {
-    return new File(getCurrentDir(), META_FILE_NAME
-        + AtomicFileOutputStream.TMP_EXTENSION);
+    return AtomicFileOutputStream.getTemporaryFile(getMetaFile());
   }
 
   File getMetaConfFile() {
     return new File(getCurrentDir(), META_FILE_NAME + CONF_EXTENSION);
   }
 
-  public File getStateMachineDir() {
-    return new File(getRoot(), STATE_MACHINE);
-  }
-
-  /** Returns a uniquely named temporary directory under $rootdir/tmp/ */
-  public File getNewTempDir() {
-    return new File(new File(getRoot(), TEMP), UUID.randomUUID().toString());
-  }
-
-  public Path relativizeToRoot(Path p) {
-    if (p.isAbsolute()) {
-      return getRoot().toPath().relativize(p);
-    }
-    return p;
-  }
-
-
   /**
    * Check to see if current/ directory is empty.
    */
@@ -187,14 +151,15 @@ public class RaftStorageDirectory {
     }
 
     // check whether current directory is valid
-    if (hasMetaFile()) {
+    if (isHealthy()) {
       return StorageState.NORMAL;
     } else {
       return StorageState.NOT_FORMATTED;
     }
   }
 
-  public boolean hasMetaFile() {
+  @Override
+  public boolean isHealthy() {
     return getMetaFile().exists();
   }
 
@@ -209,8 +174,8 @@ public class RaftStorageDirectory {
    *
    * @throws IOException if locking fails
    */
-  public void lock() throws IOException {
-    final File lockF = new File(root, STORAGE_FILE_LOCK);
+  void lock() throws IOException {
+    final File lockF = new File(root, IN_USE_LOCK_NAME);
     final FileLock newLock = FileUtils.attempt(() -> tryLock(lockF), () -> "tryLock " + lockF);
     if (newLock == null) {
       String msg = "Cannot lock storage " + this.root
@@ -273,7 +238,7 @@ public class RaftStorageDirectory {
   /**
    * Unlock storage.
    */
-  public void unlock() throws IOException {
+  void unlock() throws IOException {
     if (this.lock == null) {
       return;
     }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java
similarity index 87%
rename from ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
rename to ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java
index 9a5eb08..4fe8a7b 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java
@@ -18,16 +18,13 @@
 package org.apache.ratis.server.storage;
 
 import org.apache.ratis.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.server.RaftServerConfigKeys.Log.CorruptionPolicy;
 import org.apache.ratis.server.RaftConfiguration;
+import org.apache.ratis.server.RaftServerConfigKeys.Log.CorruptionPolicy;
 import org.apache.ratis.server.raftlog.LogProtoUtils;
-import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
+import org.apache.ratis.server.storage.RaftStorageDirectoryImpl.StorageState;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.io.Closeable;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -37,8 +34,7 @@ import java.nio.file.Files;
 import java.util.Optional;
 
 /** The storage of a {@link org.apache.ratis.server.RaftServer}. */
-public class RaftStorage implements Closeable {
-  private static final Logger LOG = LoggerFactory.getLogger(RaftStorage.class);
+public class RaftStorageImpl implements RaftStorage {
 
   public enum StartupOption {
     /** Format the storage. */
@@ -46,17 +42,17 @@ public class RaftStorage implements Closeable {
   }
 
   // TODO support multiple storage directories
-  private final RaftStorageDirectory storageDir;
+  private final RaftStorageDirectoryImpl storageDir;
   private final StorageState state;
   private final CorruptionPolicy logCorruptionPolicy;
   private volatile RaftStorageMetadataFileImpl metaFile;
 
-  public RaftStorage(File dir, CorruptionPolicy logCorruptionPolicy) throws IOException {
+  public RaftStorageImpl(File dir, CorruptionPolicy logCorruptionPolicy) throws IOException {
     this(dir, logCorruptionPolicy, null);
   }
 
-  public RaftStorage(File dir, CorruptionPolicy logCorruptionPolicy, StartupOption option) throws IOException {
-    this.storageDir = new RaftStorageDirectory(dir);
+  RaftStorageImpl(File dir, CorruptionPolicy logCorruptionPolicy, StartupOption option) throws IOException {
+    this.storageDir = new RaftStorageDirectoryImpl(dir);
     if (option == StartupOption.FORMAT) {
       if (storageDir.analyzeStorage(false) == StorageState.NON_EXISTENT) {
         throw new IOException("Cannot format " + storageDir);
@@ -118,7 +114,8 @@ public class RaftStorage implements Closeable {
     }
   }
 
-  public RaftStorageDirectory getStorageDir() {
+  @Override
+  public RaftStorageDirectoryImpl getStorageDir() {
     return storageDir;
   }
 
@@ -127,7 +124,8 @@ public class RaftStorage implements Closeable {
     storageDir.unlock();
   }
 
-  public RaftStorageMetadataFile getMetaFile() {
+  @Override
+  public RaftStorageMetadataFile getMetadataFile() {
     return metaFile;
   }
 
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
index 63985aa..cc91d8f 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.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
@@ -21,6 +21,7 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.channels.FileChannel;
+import java.util.UUID;
 
 import org.apache.ratis.io.CorruptedFileException;
 import org.apache.ratis.io.MD5Hash;
@@ -60,7 +61,8 @@ public class SnapshotManager {
     final long lastIncludedIndex = snapshotChunkRequest.getTermIndex().getIndex();
     final RaftStorageDirectory dir = storage.getStorageDir();
 
-    File tmpDir = dir.getNewTempDir();
+    // create a unique temporary directory
+    final File tmpDir =  new File(dir.getTmpDir(), UUID.randomUUID().toString());
     FileUtils.createDirectories(tmpDir);
     tmpDir.deleteOnExit();
 
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java
index 2214dc4..206ed8f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/RaftStorageTestUtils.java
@@ -33,7 +33,7 @@ import java.util.function.Consumer;
 
 public interface RaftStorageTestUtils {
   static RaftStorage newRaftStorage(File dir) throws IOException {
-    return new RaftStorage(dir, null);
+    return new RaftStorageImpl(dir, null);
   }
 
   static String getLogFlushTimeMetric(String memberId) {
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
index ce9e0b7..64410d8 100644
--- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
+++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
@@ -22,7 +22,7 @@ import static org.apache.ratis.statemachine.impl.SimpleStateMachineStorage.SNAPS
 import org.apache.ratis.BaseTest;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
+import org.apache.ratis.server.storage.RaftStorageDirectoryImpl.StorageState;
 import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
 import org.apache.ratis.statemachine.impl.SnapshotRetentionPolicy;
 import org.apache.ratis.util.FileUtils;
@@ -45,6 +45,10 @@ import java.util.regex.Matcher;
  * Test RaftStorage and RaftStorageDirectory
  */
 public class TestRaftStorage extends BaseTest {
+  static RaftStorageImpl newRaftStorage(File dir) throws IOException {
+    return new RaftStorageImpl(dir, null);
+  }
+
   private File storageDir;
 
   @Before
@@ -59,8 +63,8 @@ public class TestRaftStorage extends BaseTest {
     }
   }
 
-  static RaftStorage formatRaftStorage(File dir) throws IOException {
-    return new RaftStorage(dir, null, RaftStorage.StartupOption.FORMAT);
+  static RaftStorageImpl formatRaftStorage(File dir) throws IOException {
+    return new RaftStorageImpl(dir, null, RaftStorageImpl.StartupOption.FORMAT);
   }
 
   @Test
@@ -68,7 +72,7 @@ public class TestRaftStorage extends BaseTest {
     FileUtils.deleteFully(storageDir);
 
     // we will format the empty directory
-    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
+    final RaftStorageImpl storage = newRaftStorage(storageDir);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
 
     try {
@@ -82,7 +86,7 @@ public class TestRaftStorage extends BaseTest {
     FileUtils.deleteFully(storageDir);
     Assert.assertTrue(storageDir.createNewFile());
     try {
-      RaftStorageTestUtils.newRaftStorage(storageDir);
+      newRaftStorage(storageDir);
       Assert.fail();
     } catch (IOException e) {
       Assert.assertTrue(
@@ -95,7 +99,7 @@ public class TestRaftStorage extends BaseTest {
    */
   @Test
   public void testStorage() throws Exception {
-    RaftStorageDirectory sd = new RaftStorageDirectory(storageDir);
+    final RaftStorageDirectoryImpl sd = new RaftStorageDirectoryImpl(storageDir);
     try {
       StorageState state = sd.analyzeStorage(true);
       Assert.assertEquals(StorageState.NOT_FORMATTED, state);
@@ -104,7 +108,7 @@ public class TestRaftStorage extends BaseTest {
       sd.unlock();
     }
 
-    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
+    RaftStorageImpl storage = newRaftStorage(storageDir);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
     storage.close();
 
@@ -138,7 +142,7 @@ public class TestRaftStorage extends BaseTest {
 
   @Test
   public void testMetaFile() throws Exception {
-    RaftStorage storage = formatRaftStorage(storageDir);
+    final RaftStorageImpl storage = formatRaftStorage(storageDir);
     assertMetadataFile(storage.getStorageDir().getMetaFile());
     storage.close();
   }
@@ -148,18 +152,18 @@ public class TestRaftStorage extends BaseTest {
    */
   @Test
   public void testCleanMetaTmpFile() throws Exception {
-    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
+    RaftStorageImpl storage = newRaftStorage(storageDir);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
     storage.close();
 
-    RaftStorageDirectory sd = new RaftStorageDirectory(storageDir);
+    final RaftStorageDirectoryImpl sd = new RaftStorageDirectoryImpl(storageDir);
     File metaFile = sd.getMetaFile();
     FileUtils.move(metaFile, sd.getMetaTmpFile());
 
     Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false));
 
     try {
-      RaftStorageTestUtils.newRaftStorage(storageDir);
+      newRaftStorage(storageDir);
       Assert.fail("should throw IOException since storage dir is not formatted");
     } catch (IOException e) {
       Assert.assertTrue(
@@ -172,7 +176,7 @@ public class TestRaftStorage extends BaseTest {
     Assert.assertTrue(sd.getMetaTmpFile().createNewFile());
     Assert.assertTrue(sd.getMetaTmpFile().exists());
     try {
-      storage = RaftStorageTestUtils.newRaftStorage(storageDir);
+      storage = newRaftStorage(storageDir);
       Assert.assertEquals(StorageState.NORMAL, storage.getState());
       Assert.assertFalse(sd.getMetaTmpFile().exists());
       Assert.assertTrue(sd.getMetaFile().exists());
@@ -216,7 +220,7 @@ public class TestRaftStorage extends BaseTest {
 
 
     SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage();
-    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
+    final RaftStorage storage = newRaftStorage(storageDir);
     simpleStateMachineStorage.init(storage);
 
     List<Long> indices = new ArrayList<>();