You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sz...@apache.org on 2017/08/17 21:32:01 UTC

incubator-ratis git commit: RATIS-106. Consistently use File for storageDir.

Repository: incubator-ratis
Updated Branches:
  refs/heads/master 3e0ad68a0 -> c0c3a3748


RATIS-106. Consistently use File for storageDir.


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

Branch: refs/heads/master
Commit: c0c3a3748c83517348c7d65016c8224029d30f90
Parents: 3e0ad68
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Aug 17 10:53:16 2017 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Thu Aug 17 10:53:16 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/ratis/conf/ConfUtils.java   | 23 +++++++--
 .../org/apache/ratis/conf/RaftProperties.java   | 15 ++++++
 .../java/org/apache/ratis/util/FileUtils.java   | 49 --------------------
 .../ratis/server/RaftServerConfigKeys.java      | 11 +++--
 .../apache/ratis/server/impl/ServerState.java   |  6 ++-
 .../ratis/server/storage/RaftStorage.java       | 16 +------
 .../java/org/apache/ratis/MiniRaftCluster.java  | 22 ++++-----
 .../java/org/apache/ratis/RaftTestUtil.java     |  5 +-
 .../ratis/server/storage/TestCacheEviction.java |  4 +-
 .../server/storage/TestRaftLogReadWrite.java    | 13 +++---
 .../server/storage/TestRaftLogSegment.java      | 10 ++--
 .../ratis/server/storage/TestRaftStorage.java   | 27 ++++-------
 .../server/storage/TestSegmentedRaftLog.java    |  4 +-
 13 files changed, 86 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
index 169fb61..74f4f07 100644
--- a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.conf;
 
+import org.apache.ratis.shaded.com.google.common.base.Objects;
 import org.apache.ratis.util.CheckedBiConsumer;
 import org.apache.ratis.util.NetUtils;
 import org.apache.ratis.util.SizeInBytes;
@@ -24,6 +25,7 @@ import org.apache.ratis.util.TimeDuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.net.InetSocketAddress;
@@ -35,8 +37,8 @@ import java.util.function.Consumer;
 public interface ConfUtils {
   Logger LOG = LoggerFactory.getLogger(ConfUtils.class);
 
-  static void logGet(String key, Object value) {
-    LOG.info("{} = {}", key, value);
+  static <T> void logGet(String key, T value, T defaultValue) {
+    LOG.info("{} = {} ({})", key, value, Objects.equal(value, defaultValue)? "default": "custom");
   }
 
   static void logSet(String key, Object value) {
@@ -121,6 +123,14 @@ public interface ConfUtils {
   }
 
   @SafeVarargs
+  static File getFile(
+      BiFunction<String, File, File> fileGetter,
+      String key, File defaultValue, BiConsumer<String, File>... assertions) {
+    return get(fileGetter, key, defaultValue, assertions);
+  }
+
+
+  @SafeVarargs
   static SizeInBytes getSizeInBytes(
       BiFunction<String, SizeInBytes, SizeInBytes> getter,
       String key, SizeInBytes defaultValue, BiConsumer<String, SizeInBytes>... assertions) {
@@ -142,7 +152,7 @@ public interface ConfUtils {
   static <T> T get(BiFunction<String, T, T> getter,
       String key, T defaultValue, BiConsumer<String, T>... assertions) {
     final T value = getter.apply(key, defaultValue);
-    logGet(key, value);
+    logGet(key, value, defaultValue);
     Arrays.asList(assertions).forEach(a -> a.accept(key, value));
     return value;
   }
@@ -175,6 +185,13 @@ public interface ConfUtils {
   }
 
   @SafeVarargs
+  static void setFile(
+      BiConsumer<String, File> fileSetter, String key, File value,
+      BiConsumer<String, File>... assertions) {
+    set(fileSetter, key, value, assertions);
+  }
+
+  @SafeVarargs
   static void setSizeInBytes(
       BiConsumer<String, String> stringSetter, String key, SizeInBytes value,
       BiConsumer<String, Long>... assertions) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java b/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java
index a8dbd39..03582b9 100644
--- a/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java
+++ b/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java
@@ -632,6 +632,21 @@ public class RaftProperties {
   }
 
   /** @return property value; if it is not set, return the default value. */
+  public File getFile(String name, File defaultValue) {
+    final String valueString = getTrimmed(name);
+    return valueString == null? defaultValue: new File(valueString);
+  }
+
+  public void setFile(String name, File value) {
+    try {
+      set(name, value.getCanonicalPath());
+    } catch (IOException e) {
+      throw new IllegalArgumentException(
+          "Failed to get canonical path from file " + value + " for " + name, e);
+    }
+  }
+
+  /** @return property value; if it is not set, return the default value. */
   public SizeInBytes getSizeInBytes(String name, SizeInBytes defaultValue) {
     final String valueString = getTrimmed(name);
     return valueString == null? defaultValue: SizeInBytes.valueOf(valueString);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
index ea6e41d..714a46c 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
@@ -120,55 +120,6 @@ public class FileUtils {
   }
 
   /**
-   * Interprets the passed string as a URI. In case of error it
-   * assumes the specified string is a file.
-   *
-   * @param s the string to interpret
-   * @return the resulting URI
-   */
-  public static URI stringAsURI(String s) throws IOException {
-    URI u = null;
-    // try to make a URI
-    try {
-      u = new URI(s);
-    } catch (URISyntaxException e){
-      LOG.error("Syntax error in URI " + s
-          + ". Please check hdfs configuration.", e);
-    }
-
-    // if URI is null or scheme is undefined, then assume it's file://
-    if(u == null || u.getScheme() == null){
-      LOG.warn("Path " + s + " should be specified as a URI "
-          + "in configuration files. Please update configuration.");
-      u = fileAsURI(new File(s));
-    }
-    return u;
-  }
-
-  /**
-   * Converts the passed File to a URI. This method trims the trailing slash if
-   * one is appended because the underlying file is in fact a directory that
-   * exists.
-   *
-   * @param f the file to convert
-   * @return the resulting URI
-   */
-  public static URI fileAsURI(File f) throws IOException {
-    URI u = f.getCanonicalFile().toURI();
-
-    // trim the trailing slash, if it's present
-    if (u.getPath().endsWith("/")) {
-      String uriAsString = u.toString();
-      try {
-        u = new URI(uriAsString.substring(0, uriAsString.length() - 1));
-      } catch (URISyntaxException e) {
-        throw new IOException(e);
-      }
-    }
-    return u;
-  }
-
-  /**
    * A wrapper for {@link File#listFiles()}. This java.io API returns null
    * when a dir is not a directory or for any I/O error. Instead of having
    * null check everywhere File#listFiles() is used, we will add utility API

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
index abab5af..2586dfc 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
@@ -22,6 +22,7 @@ import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.util.TimeDuration;
 
+import java.io.File;
 import java.util.concurrent.TimeUnit;
 
 import static org.apache.ratis.conf.ConfUtils.*;
@@ -30,12 +31,12 @@ public interface RaftServerConfigKeys {
   String PREFIX = "raft.server";
 
   String STORAGE_DIR_KEY = PREFIX + ".storage.dir";
-  String STORAGE_DIR_DEFAULT = "file:///tmp/raft-server/";
-  static String storageDir(RaftProperties properties) {
-    return get(properties::getTrimmed, STORAGE_DIR_KEY, STORAGE_DIR_DEFAULT);
+  File STORAGE_DIR_DEFAULT = new File("/tmp/raft-server/");
+  static File storageDir(RaftProperties properties) {
+    return getFile(properties::getFile, STORAGE_DIR_KEY, STORAGE_DIR_DEFAULT);
   }
-  static void setStorageDir(RaftProperties properties, String storageDir) {
-    set(properties::set, STORAGE_DIR_KEY, storageDir);
+  static void setStorageDir(RaftProperties properties, File storageDir) {
+    setFile(properties::setFile, STORAGE_DIR_KEY, storageDir);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
----------------------------------------------------------------------
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 5ab1517..5db3509 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
@@ -33,6 +33,7 @@ import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.ProtoUtils;
 
 import java.io.Closeable;
+import java.io.File;
 import java.io.IOException;
 import java.util.function.Consumer;
 
@@ -84,7 +85,10 @@ public class ServerState implements Closeable {
     RaftConfiguration initialConf = RaftConfiguration.newBuilder()
         .setConf(group.getPeers()).build();
     configurationManager = new ConfigurationManager(initialConf);
-    storage = new RaftStorage(prop, group.getGroupId(), RaftServerConstants.StartupOption.REGULAR);
+
+    final File dir = RaftServerConfigKeys.storageDir(prop);
+    storage = new RaftStorage(new File(dir, group.getGroupId().toString()),
+        RaftServerConstants.StartupOption.REGULAR);
     snapshotManager = new SnapshotManager(storage, id);
 
     long lastApplied = initStatemachine(stateMachine, prop);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
----------------------------------------------------------------------
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 0b38a31..84363fe 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
@@ -17,14 +17,10 @@
  */
 package org.apache.ratis.server.storage;
 
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.RaftGroupId;
-import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachineStorage;
-import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,18 +39,8 @@ public class RaftStorage implements Closeable {
   private volatile MetaFile metaFile;
   private StateMachineStorage stateMachineStorage;
 
-  public RaftStorage(RaftProperties prop, RaftServerConstants.StartupOption option)
+  public RaftStorage(File dir, RaftServerConstants.StartupOption option)
       throws IOException {
-    this(prop, null, option);
-  }
-
-  public RaftStorage(RaftProperties prop, RaftGroupId groupId, RaftServerConstants.StartupOption option)
-      throws IOException {
-    final String dirStr = RaftServerConfigKeys.storageDir(prop);
-    File dir = new File(FileUtils.stringAsURI(dirStr).getPath());
-    if (groupId != null) {
-      dir = new File(dir, groupId.toString());
-    }
     storageDir = new RaftStorageDirectory(dir);
     if (option == RaftServerConstants.StartupOption.FORMAT) {
       if (storageDir.analyzeStorage(false) == StorageState.NON_EXISTENT) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
index becbd0d..c47966f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
+++ b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
@@ -107,15 +107,15 @@ public abstract class MiniRaftCluster {
     return new RaftGroup(RaftGroupId.createId(), peers);
   }
 
-  private static String getBaseDirectory() {
-    return System.getProperty("test.build.data", "target/test/data") + "/raft/";
+  private File getStorageDir(RaftPeerId id) {
+    return new File(RaftTestUtil.getTestBaseDirectory()
+        + "/" + getClass().getSimpleName() + "/" + id);
   }
 
-  private static void formatDir(String dirStr) {
-    final File serverDir = new File(dirStr);
-    Preconditions.assertTrue(FileUtils.fullyDelete(serverDir),
-        "Failed to format directory %s", dirStr);
-    LOG.info("Formatted directory {}", dirStr);
+  private static void formatDir(File dir) {
+    Preconditions.assertTrue(FileUtils.fullyDelete(dir),
+        "Failed to format directory %s", dir);
+    LOG.info("Formatted directory {}", dir);
   }
 
   public static String[] generateIds(int numServers, int base) {
@@ -130,7 +130,6 @@ public abstract class MiniRaftCluster {
   protected RaftGroup group;
   protected final RaftProperties properties;
   protected final Parameters parameters;
-  private final String testBaseDir;
   protected final Map<RaftPeerId, RaftServerProxy> servers = new ConcurrentHashMap<>();
 
   protected MiniRaftCluster(String[] ids, RaftProperties properties, Parameters parameters) {
@@ -141,7 +140,6 @@ public abstract class MiniRaftCluster {
     final RpcType rpcType = RaftConfigKeys.Rpc.type(properties);
     this.clientFactory = ClientFactory.cast(
         rpcType.newFactory(parameters));
-    this.testBaseDir = getBaseDirectory();
 
     ExitUtils.disableSystemExit();
   }
@@ -202,12 +200,12 @@ public abstract class MiniRaftCluster {
   private RaftServerProxy newRaftServer(RaftPeerId id, RaftGroup group,
       boolean format) {
     try {
-      final String dirStr = testBaseDir + id;
+      final File dir = getStorageDir(id);
       if (format) {
-        formatDir(dirStr);
+        formatDir(dir);
       }
       final RaftProperties prop = new RaftProperties(properties);
-      RaftServerConfigKeys.setStorageDir(prop, dirStr);
+      RaftServerConfigKeys.setStorageDir(prop, dir);
       final StateMachine stateMachine = getStateMachine4Test(properties);
       return newRaftServer(id, stateMachine, group, prop);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
index f785a30..ddeaf8f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
@@ -47,10 +47,13 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.function.BooleanSupplier;
 import java.util.function.IntSupplier;
 
-public class RaftTestUtil {
+public interface RaftTestUtil {
   public static final LogEntryProto[] EMPTY_LOGENTRY_ARRAY = new LogEntryProto[0];
   static final Logger LOG = LoggerFactory.getLogger(RaftTestUtil.class);
 
+  static String getTestBaseDirectory() {
+    return System.getProperty("test.build.data", "target/test/data");
+  }
 
   public static RaftServerImpl getImplAsUnchecked(RaftServerProxy proxy) {
     return JavaUtils.callAsUnchecked(proxy::getImpl);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
index 92885f5..9961837 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
@@ -155,8 +155,8 @@ public class TestCacheEviction {
     final int maxCachedNum = RaftServerConfigKeys.Log.maxCachedSegmentNum(prop);
 
     File storageDir = RaftTestUtil.getTestDir(TestSegmentedRaftLog.class);
-    RaftServerConfigKeys.setStorageDir(prop, storageDir.getCanonicalPath());
-    RaftStorage storage = new RaftStorage(prop, RaftServerConstants.StartupOption.REGULAR);
+    RaftServerConfigKeys.setStorageDir(prop, storageDir);
+    RaftStorage storage = new RaftStorage(storageDir, RaftServerConstants.StartupOption.REGULAR);
 
     RaftServerImpl server = Mockito.mock(RaftServerImpl.class);
     ServerState state = Mockito.mock(ServerState.class);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
index 3c10815..f1e3566 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
@@ -61,8 +61,7 @@ public class TestRaftLogReadWrite {
   public void setup() throws Exception {
     storageDir = RaftTestUtil.getTestDir(TestRaftLogReadWrite.class);
     properties = new RaftProperties();
-    RaftServerConfigKeys.setStorageDir(properties,
-        FileUtils.fileAsURI(storageDir).toString());
+    RaftServerConfigKeys.setStorageDir(properties, storageDir);
   }
 
   @After
@@ -104,7 +103,7 @@ public class TestRaftLogReadWrite {
    */
   @Test
   public void testReadWriteLog() throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
     long size = SegmentedRaftLog.HEADER_BYTES.length;
 
@@ -125,7 +124,7 @@ public class TestRaftLogReadWrite {
 
   @Test
   public void testAppendLog() throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
     LogEntryProto[] entries = new LogEntryProto[200];
     try (LogOutputStream out =
@@ -161,7 +160,7 @@ public class TestRaftLogReadWrite {
    */
   @Test
   public void testReadWithPadding() throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
     long size = SegmentedRaftLog.HEADER_BYTES.length;
 
@@ -193,7 +192,7 @@ public class TestRaftLogReadWrite {
     RaftServerConfigKeys.Log.setPreallocatedSize(properties, SizeInBytes.valueOf("4MB"));
     RaftServerConfigKeys.Log.setSegmentSizeMax(properties, SizeInBytes.valueOf("16MB"));
 
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
 
     LogEntryProto[] entries = new LogEntryProto[10];
@@ -242,7 +241,7 @@ public class TestRaftLogReadWrite {
    */
   @Test
   public void testReadWithEntryCorruption() throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     File openSegment = storage.getStorageDir().getOpenLogFile(0);
     try (LogOutputStream out =
              new LogOutputStream(openSegment, false, properties)) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
index 73709fc..769f923 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
@@ -57,7 +57,7 @@ public class TestRaftLogSegment {
   @Before
   public void setup() throws Exception {
     storageDir = RaftTestUtil.getTestDir(TestRaftLogSegment.class);
-    RaftServerConfigKeys.setStorageDir(properties, storageDir.getCanonicalPath());
+    RaftServerConfigKeys.setStorageDir(properties, storageDir);
   }
 
   @After
@@ -69,7 +69,7 @@ public class TestRaftLogSegment {
 
   private File prepareLog(boolean isOpen, long start, int size, long term)
       throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     File file = isOpen ? storage.getStorageDir().getOpenLogFile(start) :
         storage.getStorageDir().getClosedLogFile(start, start + size - 1);
 
@@ -120,7 +120,7 @@ public class TestRaftLogSegment {
   private void testLoadSegment(boolean loadInitial) throws Exception {
     // load an open segment
     File openSegmentFile = prepareLog(true, 0, 100, 0);
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, 0,
         INVALID_LOG_INDEX, true, loadInitial, null);
     checkLogSegment(openSegment, 0, 99, true, openSegmentFile.length(), 0);
@@ -237,7 +237,7 @@ public class TestRaftLogSegment {
 
   @Test
   public void testPreallocateSegment() throws Exception {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     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,
@@ -292,7 +292,7 @@ public class TestRaftLogSegment {
     RaftServerConfigKeys.Log.setSegmentSizeMax(properties, max);
     RaftServerConfigKeys.Log.setPreallocatedSize(properties, SizeInBytes.valueOf("16KB"));
     RaftServerConfigKeys.Log.setWriteBufferSize(properties, SizeInBytes.valueOf("10KB"));
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     final File file = storage.getStorageDir().getOpenLogFile(0);
 
     final byte[] content = new byte[1024];

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
index ede9103..d8ca627 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
@@ -18,14 +18,9 @@
 package org.apache.ratis.server.storage;
 
 import org.apache.ratis.RaftTestUtil;
-import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.io.nativeio.NativeIO;
-import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.server.storage.MetaFile;
-import org.apache.ratis.server.storage.RaftStorage;
-import org.apache.ratis.server.storage.RaftStorageDirectory;
 import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
 import org.apache.ratis.statemachine.SimpleStateMachineStorage;
 import org.apache.ratis.util.FileUtils;
@@ -44,12 +39,10 @@ import java.util.concurrent.ThreadLocalRandom;
  */
 public class TestRaftStorage {
   private File storageDir;
-  private final RaftProperties properties = new RaftProperties();
 
   @Before
   public void setup() throws Exception {
     storageDir = RaftTestUtil.getTestDir(TestRaftStorage.class);
-    RaftServerConfigKeys.setStorageDir(properties, storageDir.getCanonicalPath());
   }
 
   @After
@@ -64,11 +57,11 @@ public class TestRaftStorage {
     FileUtils.fullyDelete(storageDir);
 
     // we will format the empty directory
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
 
     try {
-      new RaftStorage(properties, StartupOption.FORMAT).close();
+      new RaftStorage(storageDir, StartupOption.FORMAT).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"));
@@ -78,7 +71,7 @@ public class TestRaftStorage {
     FileUtils.fullyDelete(storageDir);
     Assert.assertTrue(storageDir.createNewFile());
     try {
-      new RaftStorage(properties, StartupOption.REGULAR);
+      new RaftStorage(storageDir, StartupOption.REGULAR);
       Assert.fail();
     } catch (IOException e) {
       Assert.assertTrue(
@@ -100,7 +93,7 @@ public class TestRaftStorage {
       sd.unlock();
     }
 
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
     storage.close();
 
@@ -122,7 +115,7 @@ public class TestRaftStorage {
     Assert.assertEquals("peer1", metaFile.getVotedFor());
 
     // test format
-    storage = new RaftStorage(properties, StartupOption.FORMAT);
+    storage = new RaftStorage(storageDir, StartupOption.FORMAT);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
     metaFile = new MetaFile(sd.getMetaFile());
     Assert.assertEquals(MetaFile.DEFAULT_TERM, metaFile.getTerm());
@@ -132,7 +125,7 @@ public class TestRaftStorage {
 
   @Test
   public void testMetaFile() throws Exception {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.FORMAT);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.FORMAT);
     File m = storage.getStorageDir().getMetaFile();
     Assert.assertTrue(m.exists());
     MetaFile metaFile = new MetaFile(m);
@@ -157,7 +150,7 @@ public class TestRaftStorage {
    */
   @Test
   public void testCleanMetaTmpFile() throws Exception {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
     Assert.assertEquals(StorageState.NORMAL, storage.getState());
     storage.close();
 
@@ -168,7 +161,7 @@ public class TestRaftStorage {
     Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false));
 
     try {
-      new RaftStorage(properties, StartupOption.REGULAR);
+      new RaftStorage(storageDir, StartupOption.REGULAR);
       Assert.fail("should throw IOException since storage dir is not formatted");
     } catch (IOException e) {
       Assert.assertTrue(
@@ -176,12 +169,12 @@ public class TestRaftStorage {
     }
 
     // let the storage dir contain both raft-meta and raft-meta.tmp
-    new RaftStorage(properties, StartupOption.FORMAT).close();
+    new RaftStorage(storageDir, StartupOption.FORMAT).close();
     Assert.assertTrue(sd.getMetaFile().exists());
     Assert.assertTrue(sd.getMetaTmpFile().createNewFile());
     Assert.assertTrue(sd.getMetaTmpFile().exists());
     try {
-      storage = new RaftStorage(properties, StartupOption.REGULAR);
+      storage = new RaftStorage(storageDir, StartupOption.REGULAR);
       Assert.assertEquals(StorageState.NORMAL, storage.getState());
       Assert.assertFalse(sd.getMetaTmpFile().exists());
       Assert.assertTrue(sd.getMetaFile().exists());

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c0c3a374/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
index 1db2fe5..811ba6d 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
@@ -76,8 +76,8 @@ public class TestSegmentedRaftLog {
   public void setup() throws Exception {
     storageDir = RaftTestUtil.getTestDir(TestSegmentedRaftLog.class);
     properties = new RaftProperties();
-    RaftServerConfigKeys.setStorageDir(properties, storageDir.getCanonicalPath());
-    storage = new RaftStorage(properties, RaftServerConstants.StartupOption.REGULAR);
+    RaftServerConfigKeys.setStorageDir(properties, storageDir);
+    storage = new RaftStorage(storageDir, RaftServerConstants.StartupOption.REGULAR);
   }
 
   @After