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/09 12:52:30 UTC

[incubator-ratis] branch master updated: RATIS-1225. Remove RaftServerConstants. (#342)

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 4e32dd7  RATIS-1225. Remove RaftServerConstants. (#342)
4e32dd7 is described below

commit 4e32dd78758b811f0773e55976d8a9628767e085
Author: Tsz-Wo Nicholas Sze <sz...@apache.org>
AuthorDate: Wed Dec 9 20:52:23 2020 +0800

    RATIS-1225. Remove RaftServerConstants. (#342)
---
 .../ratis/server/impl/RaftServerConstants.java     | 44 ----------------------
 .../org/apache/ratis/server/impl/ServerState.java  |  3 +-
 .../apache/ratis/server/storage/RaftStorage.java   | 20 ++++++----
 .../ratis/server/storage/RaftStorageTestUtils.java |  5 +++
 .../raftlog/segmented/TestCacheEviction.java       |  4 +-
 .../server/raftlog/segmented/TestLogSegment.java   | 14 +++----
 .../raftlog/segmented/TestRaftLogReadWrite.java    | 12 +++---
 .../raftlog/segmented/TestSegmentedRaftLog.java    |  4 +-
 .../ratis/server/storage/TestRaftStorage.java      | 27 +++++++------
 9 files changed, 50 insertions(+), 83 deletions(-)

diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerConstants.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerConstants.java
deleted file mode 100644
index bf1e9db..0000000
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerConstants.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ratis.server.impl;
-
-public final class RaftServerConstants {
-  private RaftServerConstants() {
-    //Never constructed
-  }
-
-  public enum StartupOption {
-    FORMAT("format"),
-    REGULAR("regular");
-
-    private final String option;
-
-    StartupOption(String arg) {
-      this.option = arg;
-    }
-
-    public static StartupOption getOption(String arg) {
-      for (StartupOption s : StartupOption.values()) {
-        if (s.option.equals(arg)) {
-          return s;
-        }
-      }
-      return REGULAR;
-    }
-  }
-}
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 2d62494..88189d6 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
@@ -107,8 +107,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, RaftServerConstants.StartupOption.REGULAR,
-        RaftServerConfigKeys.Log.corruptionPolicy(prop));
+    storage = new RaftStorage(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/storage/RaftStorage.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
index 18810f3..785d725 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/RaftStorage.java
@@ -20,7 +20,6 @@ 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.impl.RaftServerConstants;
 import org.apache.ratis.server.impl.ServerProtoUtils;
 import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
 import org.apache.ratis.util.JavaUtils;
@@ -34,25 +33,30 @@ import java.io.IOException;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 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 enum StartupOption {
+    /** Format the storage. */
+    FORMAT;
+  }
+
   // TODO support multiple storage directories
   private final RaftStorageDirectory storageDir;
   private final StorageState state;
   private final CorruptionPolicy logCorruptionPolicy;
   private volatile MetaFile metaFile;
 
-  public RaftStorage(File dir, RaftServerConstants.StartupOption option)
-      throws IOException {
-    this(dir, option, CorruptionPolicy.getDefault());
+  public RaftStorage(File dir, CorruptionPolicy logCorruptionPolicy) throws IOException {
+    this(dir, logCorruptionPolicy, null);
   }
 
-  public RaftStorage(File dir, RaftServerConstants.StartupOption option, CorruptionPolicy logCorruptionPolicy)
-      throws IOException {
+  public RaftStorage(File dir, CorruptionPolicy logCorruptionPolicy, StartupOption option) throws IOException {
     this.storageDir = new RaftStorageDirectory(dir);
-    if (option == RaftServerConstants.StartupOption.FORMAT) {
+    if (option == StartupOption.FORMAT) {
       if (storageDir.analyzeStorage(false) == StorageState.NON_EXISTENT) {
         throw new IOException("Cannot format " + storageDir);
       }
@@ -68,7 +72,7 @@ public class RaftStorage implements Closeable {
             + ". Its state: " + state);
       }
     }
-    this.logCorruptionPolicy = logCorruptionPolicy;
+    this.logCorruptionPolicy = Optional.ofNullable(logCorruptionPolicy).orElseGet(CorruptionPolicy::getDefault);
   }
 
   StorageState getState() {
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 74ed663..67509e0 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
@@ -28,9 +28,14 @@ import org.apache.ratis.server.raftlog.RaftLog;
 import org.apache.ratis.server.raftlog.RaftLogIOException;
 import org.apache.ratis.util.AutoCloseableLock;
 
+import java.io.File;
+import java.io.IOException;
 import java.util.function.Consumer;
 
 public interface RaftStorageTestUtils {
+  static RaftStorage newRaftStorage(File dir) throws IOException {
+    return new RaftStorage(dir, null);
+  }
 
   static String getLogFlushTimeMetric(String memberId) {
     return getRaftLogFullMetric(memberId, RAFT_LOG_FLUSH_TIME);
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 22fabc0..a7f23fd 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
@@ -27,7 +27,6 @@ import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.DivisionInfo;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.MiniRaftCluster;
-import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.server.impl.ServerProtoUtils;
 import org.apache.ratis.server.raftlog.RaftLog;
@@ -35,6 +34,7 @@ import org.apache.ratis.server.raftlog.segmented.CacheInvalidationPolicy.CacheIn
 import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogCache.LogSegmentList;
 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.StateMachine;
 import org.apache.ratis.util.JavaUtils;
@@ -163,7 +163,7 @@ public class TestCacheEviction extends BaseTest {
 
     File storageDir = getTestDir();
     RaftServerConfigKeys.setStorageDir(prop,  Collections.singletonList(storageDir));
-    RaftStorage storage = new RaftStorage(storageDir, RaftServerConstants.StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
 
     final DivisionInfo info = Mockito.mock(DivisionInfo.class);
     Mockito.when(info.getLastAppliedIndex()).thenReturn(0L);
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java
index 0380cae..5920f8b 100644
--- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java
+++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java
@@ -21,7 +21,6 @@ import org.apache.ratis.BaseTest;
 import org.apache.ratis.RaftTestUtil.SimpleOperation;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
 import org.apache.ratis.server.impl.ServerProtoUtils;
 import org.apache.ratis.server.metrics.RaftLogMetrics;
 import org.apache.ratis.server.protocol.TermIndex;
@@ -29,6 +28,7 @@ import org.apache.ratis.server.storage.RaftStorage;
 import org.apache.ratis.server.storage.RaftStorageDirectory;
 import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto;
+import org.apache.ratis.server.storage.RaftStorageTestUtils;
 import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.Preconditions;
@@ -88,7 +88,7 @@ public class TestLogSegment extends BaseTest {
     if (!isOpen) {
       Preconditions.assertTrue(!isLastEntryPartiallyWritten, "For closed log, the last entry cannot be partially written.");
     }
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     final File file = isOpen ?
         storage.getStorageDir().getOpenLogFile(startIndex) :
         storage.getStorageDir().getClosedLogFile(startIndex, startIndex + numEntries - 1);
@@ -169,7 +169,7 @@ public class TestLogSegment extends BaseTest {
   private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWritten) throws Exception {
     // load an open segment
     final File openSegmentFile = prepareLog(true, 0, 100, 0, isLastEntryPartiallyWritten);
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, 0,
         INVALID_LOG_INDEX, true, loadInitial, null, null);
     final int delta = isLastEntryPartiallyWritten? 1: 0;
@@ -214,7 +214,7 @@ public class TestLogSegment extends BaseTest {
     RaftLogMetrics raftLogMetrics = new RaftLogMetrics("test");
 
     final File openSegmentFile = prepareLog(true, 0, 100, 0, true);
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, 0,
         INVALID_LOG_INDEX, true, true, null, raftLogMetrics);
     checkLogSegment(openSegment, 0, 98, true, openSegmentFile.length(), 0);
@@ -284,7 +284,7 @@ public class TestLogSegment extends BaseTest {
 
   @Test
   public void testPreallocateSegment() throws Exception {
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     final File file = storage.getStorageDir().getOpenLogFile(0);
     final int[] maxSizes = new int[]{1024, 1025, 1024 * 1024 - 1, 1024 * 1024,
         1024 * 1024 + 1, 2 * 1024 * 1024 - 1, 2 * 1024 * 1024,
@@ -333,7 +333,7 @@ public class TestLogSegment extends BaseTest {
   @Test
   public void testPreallocationAndAppend() throws Exception {
     final SizeInBytes max = SizeInBytes.valueOf(2, TraditionalBinaryPrefix.MEGA);
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     final File file = storage.getStorageDir().getOpenLogFile(0);
 
     final byte[] content = new byte[1024];
@@ -363,7 +363,7 @@ public class TestLogSegment extends BaseTest {
 
   @Test
   public void testZeroSizeInProgressFile() throws Exception {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     final File file = storage.getStorageDir().getOpenLogFile(0);
     storage.close();
 
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java
index 3928671..8237600 100644
--- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java
+++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java
@@ -22,10 +22,10 @@ import org.apache.ratis.RaftTestUtil.SimpleOperation;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.exceptions.ChecksumException;
 import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
 import org.apache.ratis.server.impl.ServerProtoUtils;
 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.thirdparty.com.google.protobuf.CodedOutputStream;
 import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
@@ -102,7 +102,7 @@ public class TestRaftLogReadWrite extends BaseTest {
    */
   @Test
   public void testReadWriteLog() throws IOException {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
     long size = SegmentedRaftLogFormat.getHeaderLength();
 
@@ -122,7 +122,7 @@ public class TestRaftLogReadWrite extends BaseTest {
 
   @Test
   public void testAppendLog() throws IOException {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
     LogEntryProto[] entries = new LogEntryProto[200];
     try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(openSegment, false,
@@ -155,7 +155,7 @@ public class TestRaftLogReadWrite extends BaseTest {
    */
   @Test
   public void testReadWithPadding() throws IOException {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
     long size = SegmentedRaftLogFormat.getHeaderLength();
 
@@ -184,7 +184,7 @@ public class TestRaftLogReadWrite extends BaseTest {
    */
   @Test
   public void testReadWithCorruptPadding() throws IOException {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
 
     LogEntryProto[] entries = new LogEntryProto[10];
@@ -233,7 +233,7 @@ public class TestRaftLogReadWrite extends BaseTest {
    */
   @Test
   public void testReadWithEntryCorruption() throws IOException {
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
     try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(openSegment, false,
         segmentMaxSize, preallocatedSize, ByteBuffer.allocateDirect(bufferSize))) {
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 f56fc22..59557e9 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
@@ -29,7 +29,6 @@ import org.apache.ratis.protocol.RaftGroupMemberId;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.protocol.exceptions.TimeoutIOException;
 import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.impl.RetryCacheTestUtil;
 import org.apache.ratis.server.RetryCache;
 import org.apache.ratis.server.impl.ServerProtoUtils;
@@ -38,6 +37,7 @@ import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 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.StateMachine;
 import org.apache.ratis.statemachine.impl.BaseStateMachine;
@@ -128,7 +128,7 @@ public class TestSegmentedRaftLog extends BaseTest {
     storageDir = getTestDir();
     properties = new RaftProperties();
     RaftServerConfigKeys.setStorageDir(properties,  Collections.singletonList(storageDir));
-    storage = new RaftStorage(storageDir, RaftServerConstants.StartupOption.REGULAR);
+    storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     this.segmentMaxSize =
         RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize();
     this.preallocatedSize =
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 86c8cb7..2816d3b 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
@@ -20,7 +20,6 @@ package org.apache.ratis.server.storage;
 import static org.apache.ratis.statemachine.impl.SimpleStateMachineStorage.SNAPSHOT_REGEX;
 
 import org.apache.ratis.BaseTest;
-import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
 import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage;
@@ -58,16 +57,20 @@ public class TestRaftStorage extends BaseTest {
     }
   }
 
+  static RaftStorage formatRaftStorage(File dir) throws IOException {
+    return new RaftStorage(dir, null, RaftStorage.StartupOption.FORMAT);
+  }
+
   @Test
   public void testNotExistent() throws IOException {
     FileUtils.deleteFully(storageDir);
 
     // we will format the empty directory
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
 
     try {
-      new RaftStorage(storageDir, StartupOption.FORMAT).close();
+      formatRaftStorage(storageDir).close();
       Assert.fail("the format should fail since the storage is still locked");
     } catch (IOException e) {
       Assert.assertTrue(e.getMessage().contains("directory is already locked"));
@@ -77,7 +80,7 @@ public class TestRaftStorage extends BaseTest {
     FileUtils.deleteFully(storageDir);
     Assert.assertTrue(storageDir.createNewFile());
     try {
-      new RaftStorage(storageDir, StartupOption.REGULAR);
+      RaftStorageTestUtils.newRaftStorage(storageDir);
       Assert.fail();
     } catch (IOException e) {
       Assert.assertTrue(
@@ -99,7 +102,7 @@ public class TestRaftStorage extends BaseTest {
       sd.unlock();
     }
 
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
     storage.close();
 
@@ -121,7 +124,7 @@ public class TestRaftStorage extends BaseTest {
     Assert.assertEquals("peer1", metaFile.getVotedFor());
 
     // test format
-    storage = new RaftStorage(storageDir, StartupOption.FORMAT);
+    storage = formatRaftStorage(storageDir);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
     metaFile = new MetaFile(sd.getMetaFile());
     Assert.assertEquals(MetaFile.DEFAULT_TERM, metaFile.getTerm());
@@ -131,7 +134,7 @@ public class TestRaftStorage extends BaseTest {
 
   @Test
   public void testMetaFile() throws Exception {
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.FORMAT);
+    RaftStorage storage = formatRaftStorage(storageDir);
     File m = storage.getStorageDir().getMetaFile();
     Assert.assertTrue(m.exists());
     MetaFile metaFile = new MetaFile(m);
@@ -156,7 +159,7 @@ public class TestRaftStorage extends BaseTest {
    */
   @Test
   public void testCleanMetaTmpFile() throws Exception {
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
     storage.close();
 
@@ -167,7 +170,7 @@ public class TestRaftStorage extends BaseTest {
     Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false));
 
     try {
-      new RaftStorage(storageDir, StartupOption.REGULAR);
+      RaftStorageTestUtils.newRaftStorage(storageDir);
       Assert.fail("should throw IOException since storage dir is not formatted");
     } catch (IOException e) {
       Assert.assertTrue(
@@ -175,12 +178,12 @@ public class TestRaftStorage extends BaseTest {
     }
 
     // let the storage dir contain both raft-meta and raft-meta.tmp
-    new RaftStorage(storageDir, StartupOption.FORMAT).close();
+    formatRaftStorage(storageDir).close();
     Assert.assertTrue(sd.getMetaFile().exists());
     Assert.assertTrue(sd.getMetaTmpFile().createNewFile());
     Assert.assertTrue(sd.getMetaTmpFile().exists());
     try {
-      storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+      storage = RaftStorageTestUtils.newRaftStorage(storageDir);
       Assert.assertEquals(StorageState.NORMAL, storage.getState());
       Assert.assertFalse(sd.getMetaTmpFile().exists());
       Assert.assertTrue(sd.getMetaFile().exists());
@@ -224,7 +227,7 @@ public class TestRaftStorage extends BaseTest {
 
 
     SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage();
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir);
     simpleStateMachineStorage.init(storage);
 
     List<Long> indices = new ArrayList<>();