You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2017/07/15 00:36:29 UTC

hadoop git commit: HDFS-11989. Ozone: add TestKeysRatis, TestBucketsRatis and TestVolumeRatis. Contributed by Tsz Wo Nicholas Sze.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 c99d2890e -> 90f1d5854


HDFS-11989. Ozone: add TestKeysRatis, TestBucketsRatis and TestVolumeRatis. Contributed by Tsz Wo Nicholas Sze.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/90f1d585
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/90f1d585
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/90f1d585

Branch: refs/heads/HDFS-7240
Commit: 90f1d585466cef890783af61535de1bbea1540a0
Parents: c99d289
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Jul 14 17:33:05 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Jul 14 17:33:05 2017 -0700

----------------------------------------------------------------------
 .../server/datanode/ObjectStoreHandler.java     |  3 +-
 .../apache/hadoop/ozone/OzoneConfiguration.java |  7 ++
 .../hadoop/ozone/web/client/OzoneBucket.java    | 13 ++-
 .../ozone/web/handlers/KeyProcessTemplate.java  |  7 ++
 .../web/handlers/VolumeProcessTemplate.java     |  5 +-
 .../apache/hadoop/ozone/MiniOzoneCluster.java   | 12 +++
 .../apache/hadoop/ozone/RatisTestHelper.java    | 80 ++++++++++++++--
 .../hadoop/ozone/web/client/TestBuckets.java    | 30 +++++-
 .../ozone/web/client/TestBucketsRatis.java      | 75 +++++++++++++++
 .../hadoop/ozone/web/client/TestKeys.java       | 83 +++++++++++-----
 .../hadoop/ozone/web/client/TestKeysRatis.java  | 88 +++++++++++++++++
 .../hadoop/ozone/web/client/TestVolume.java     | 74 ++++++++++++---
 .../ozone/web/client/TestVolumeRatis.java       | 99 ++++++++++++++++++++
 .../src/test/resources/log4j.properties         |  2 +-
 14 files changed, 527 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
index ab89ede..40c7b1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
@@ -126,7 +126,8 @@ public final class ObjectStoreHandler implements Closeable {
               NetUtils.getDefaultSocketFactory(conf),
               Client.getRpcTimeout(conf)));
 
-      storageHandler = new DistributedStorageHandler(new OzoneConfiguration(),
+      storageHandler = new DistributedStorageHandler(
+          new OzoneConfiguration(conf),
           this.storageContainerLocationClient,
           this.keySpaceManagerClient);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
index 70efa49..a16e8d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
@@ -33,4 +33,11 @@ public class OzoneConfiguration extends Configuration {
     Configuration.addDefaultResource("ozone-default.xml");
     Configuration.addDefaultResource("ozone-site.xml");
   }
+
+  public OzoneConfiguration() {
+  }
+
+  public OzoneConfiguration(Configuration conf) {
+    super(conf);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
index 08587eb..b973095 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
@@ -40,6 +40,8 @@ import org.apache.http.entity.FileEntity;
 import org.apache.http.entity.InputStreamEntity;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Strings;
 
@@ -65,6 +67,7 @@ import static org.apache.hadoop.ozone.web.utils.OzoneUtils.ENCODING_NAME;
  * A Bucket class the represents an Ozone Bucket.
  */
 public class OzoneBucket {
+  static final Logger LOG = LoggerFactory.getLogger(OzoneBucket.class);
 
   private BucketInfo bucketInfo;
   private OzoneVolume volume;
@@ -236,7 +239,11 @@ public class OzoneBucket {
       executePutKey(putRequest, httpClient);
 
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage());
+      final OzoneRestClientException orce = new OzoneRestClientException(
+          "Failed to putKey: keyName=" + keyName + ", file=" + file);
+      orce.initCause(ex);
+      LOG.trace("", orce);
+      throw orce;
     } finally {
       IOUtils.closeStream(fis);
       OzoneClientUtils.releaseConnection(putRequest);
@@ -478,7 +485,9 @@ public class OzoneBucket {
         builder.addParameter(Header.OZONE_LIST_QUERY_PREFIX, prefix);
       }
 
-      getRequest = client.getHttpGet(builder.toString());
+      final String uri = builder.toString();
+      getRequest = client.getHttpGet(uri);
+      LOG.trace("listKeys URI={}", uri);
       return executeListKeys(getRequest, httpClient);
 
     } catch (IOException | URISyntaxException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java
index fed44a1..015fcf9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.handlers;
 
 import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.headers.Header;
@@ -96,6 +97,12 @@ public abstract class KeyProcessTemplate {
       LOG.debug("Invalid bucket in key call. ex:{}", argExp);
       throw newError(INVALID_BUCKET_NAME, userArgs, argExp);
     } catch (IOException fsExp) {
+      // Map KEY_NOT_FOUND to INVALID_KEY
+      if (fsExp.getMessage().endsWith(
+          KeySpaceManagerProtocolProtos.Status.KEY_NOT_FOUND.name())) {
+        throw ErrorTable.newError(ErrorTable.INVALID_KEY, userArgs, fsExp);
+      }
+
       // TODO : Handle errors from the FileSystem , let us map to server error
       // for now.
       LOG.debug("IOException. ex : {}", fsExp);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java
index 4aae67b..9bd8aa9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.handlers;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
@@ -131,7 +132,9 @@ public abstract class VolumeProcessTemplate {
                                  IOException fsExp) throws OzoneException {
     OzoneException exp = null;
 
-    if (fsExp instanceof FileAlreadyExistsException) {
+    if ((fsExp != null && fsExp.getMessage().endsWith(
+        KeySpaceManagerProtocolProtos.Status.VOLUME_ALREADY_EXISTS.name()))
+        || fsExp instanceof FileAlreadyExistsException) {
       exp = ErrorTable
           .newError(ErrorTable.VOLUME_ALREADY_EXISTS, reqID, volume, hostName);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
index 9e3c1e4..a0dd6bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
 import org.apache.hadoop.ozone.ksm.KeySpaceManager;
 import org.apache.hadoop.ozone.web.client.OzoneRestClient;
+import org.apache.hadoop.ozone.scm.ratis.RatisManager;
 import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.scm.protocolPB
     .StorageContainerLocationProtocolClientSideTranslatorPB;
@@ -75,6 +76,8 @@ public final class MiniOzoneCluster extends MiniDFSCluster
   private final KeySpaceManager ksm;
   private final Path tempPath;
 
+  private final RatisManager ratisManager;
+
   /**
    * Creates a new MiniOzoneCluster.
    *
@@ -90,6 +93,15 @@ public final class MiniOzoneCluster extends MiniDFSCluster
     this.scm = scm;
     this.ksm = ksm;
     tempPath = Paths.get(builder.getPath(), builder.getRunID());
+
+    final boolean useRatis = conf.getBoolean(
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
+        OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
+    this.ratisManager = useRatis? RatisManager.newRatisManager(conf): null;
+  }
+
+  public RatisManager getRatisManager() {
+    return ratisManager;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
index 89664eb..811b8a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
@@ -19,13 +19,19 @@
 package org.apache.hadoop.ozone;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.scm.XceiverClientRatis;
-import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.web.client.OzoneRestClient;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.ratis.rpc.RpcType;
+import org.apache.ratis.rpc.SupportedRpcType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Closeable;
 import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.stream.Collectors;
 
 /**
  * Helpers for Ratis tests.
@@ -33,6 +39,56 @@ import java.io.IOException;
 public interface RatisTestHelper {
   Logger LOG = LoggerFactory.getLogger(RatisTestHelper.class);
 
+  /** For testing Ozone with Ratis. */
+  class RatisTestSuite implements Closeable {
+    static final RpcType RPC = SupportedRpcType.NETTY;
+    static final int NUM_DATANODES = 3;
+
+    private final OzoneConfiguration conf;
+    private final MiniOzoneCluster cluster;
+
+    /**
+     * Create a {@link MiniOzoneCluster} for testing by setting
+     *   OZONE_ENABLED = true,
+     *   RATIS_ENABLED = true, and
+     *   OZONE_HANDLER_TYPE_KEY = "distributed".
+     */
+    public RatisTestSuite(final Class<?> clazz) throws IOException {
+      conf = newOzoneConfiguration(clazz, RPC);
+      cluster = newMiniOzoneCluster(NUM_DATANODES, conf);
+    }
+
+    public OzoneConfiguration getConf() {
+      return conf;
+    }
+
+    public MiniOzoneCluster getCluster() {
+      return cluster;
+    }
+
+    public int getDatanodeInfoPort() {
+      return cluster.getDataNodes().get(0).getInfoPort();
+    }
+
+    public OzoneRestClient newOzoneRestClient()
+        throws OzoneException, URISyntaxException {
+      return RatisTestHelper.newOzoneRestClient(getDatanodeInfoPort());
+    }
+
+    @Override
+    public void close() {
+      cluster.close();
+    }
+  }
+
+  static OzoneConfiguration newOzoneConfiguration(
+      Class<?> clazz, RpcType rpc) {
+    final OzoneConfiguration conf = new OzoneConfiguration();
+    ContainerTestHelper.setOzoneLocalStorageRoot(clazz, conf);
+    initRatisConf(rpc, conf);
+    return conf;
+  }
+
   static void initRatisConf(RpcType rpc, Configuration conf) {
     conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, true);
     conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY, rpc.name());
@@ -40,10 +96,20 @@ public interface RatisTestHelper {
         + " = " + rpc.name());
   }
 
-  static XceiverClientRatis newXceiverClientRatis(
-      RpcType rpcType, Pipeline pipeline, OzoneConfiguration conf)
-      throws IOException {
-    initRatisConf(rpcType, conf);
-    return XceiverClientRatis.newXceiverClientRatis(pipeline, conf);
+  static MiniOzoneCluster newMiniOzoneCluster(
+      int numDatanodes, OzoneConfiguration conf) throws IOException {
+    final MiniOzoneCluster cluster = new MiniOzoneCluster.Builder(conf)
+        .numDataNodes(numDatanodes)
+        .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
+    cluster.getRatisManager().createRatisCluster("ratis0",
+        cluster.getDataNodes().stream()
+            .map(DataNode::getDatanodeId)
+            .collect(Collectors.toList()));
+    return cluster;
+  }
+
+  static OzoneRestClient newOzoneRestClient(int port)
+      throws OzoneException, URISyntaxException {
+    return new OzoneRestClient("http://localhost:" + port);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
index 5e8f6de..0be5df6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
@@ -49,7 +49,7 @@ public class TestBuckets {
   public Timeout testTimeout = new Timeout(300000);
 
   private static MiniOzoneCluster cluster = null;
-  private static OzoneRestClient client = null;
+  private static OzoneRestClient ozoneRestClient = null;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -75,7 +75,8 @@ public class TestBuckets {
         .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
     DataNode dataNode = cluster.getDataNodes().get(0);
     final int port = dataNode.getInfoPort();
-    client = new OzoneRestClient(String.format("http://localhost:%d", port));
+    ozoneRestClient = new OzoneRestClient(
+        String.format("http://localhost:%d", port));
   }
 
   /**
@@ -90,6 +91,11 @@ public class TestBuckets {
 
   @Test
   public void testCreateBucket() throws OzoneException, IOException {
+    runTestCreateBucket(ozoneRestClient);
+  }
+
+  static void runTestCreateBucket(OzoneRestClient client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth("hdfs");
     OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
@@ -113,6 +119,11 @@ public class TestBuckets {
 
   @Test
   public void testAddBucketAcls() throws OzoneException, IOException {
+    runTestAddBucketAcls(ozoneRestClient);
+  }
+
+  static void runTestAddBucketAcls(OzoneRestClient client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth("hdfs");
     OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
@@ -127,6 +138,11 @@ public class TestBuckets {
 
   @Test
   public void testRemoveBucketAcls() throws OzoneException, IOException {
+    runTestRemoveBucketAcls(ozoneRestClient);
+  }
+
+  static void runTestRemoveBucketAcls(OzoneRestClient client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth("hdfs");
     OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
@@ -144,6 +160,11 @@ public class TestBuckets {
 
   @Test
   public void testDeleteBucket() throws OzoneException, IOException {
+    runTestDeleteBucket(ozoneRestClient);
+  }
+
+  static void runTestDeleteBucket(OzoneRestClient client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth("hdfs");
     OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
@@ -163,6 +184,11 @@ public class TestBuckets {
 
   @Test
   public void testListBucket() throws OzoneException, IOException {
+    runTestListBucket(ozoneRestClient);
+  }
+
+  static void runTestListBucket(OzoneRestClient client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth("hdfs");
     OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
new file mode 100644
index 0000000..5a27434
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
@@ -0,0 +1,75 @@
+/*
+ * 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.hadoop.ozone.web.client;
+
+import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+
+/** The same as {@link TestBuckets} except that this test is Ratis enabled. */
+public class TestBucketsRatis {
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static RatisTestHelper.RatisTestSuite suite;
+  private static OzoneRestClient ozoneRestClient;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
+    ozoneRestClient = suite.newOzoneRestClient();
+  }
+
+  @AfterClass
+  public static void shutdown() {
+    if (suite != null) {
+      suite.close();
+    }
+  }
+
+  @Test
+  public void testCreateBucket() throws OzoneException, IOException {
+    TestBuckets.runTestCreateBucket(ozoneRestClient);
+  }
+
+  @Test
+  public void testAddBucketAcls() throws OzoneException, IOException {
+    TestBuckets.runTestAddBucketAcls(ozoneRestClient);
+  }
+
+  @Test
+  public void testRemoveBucketAcls() throws OzoneException, IOException {
+    TestBuckets.runTestRemoveBucketAcls(ozoneRestClient);
+  }
+
+  @Test
+  public void testDeleteBucket() throws OzoneException, IOException {
+    TestBuckets.runTestDeleteBucket(ozoneRestClient);
+  }
+
+  @Test
+  public void testListBucket() throws OzoneException, IOException {
+    TestBuckets.runTestListBucket(ozoneRestClient);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index 1e14163..dbd7f39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -59,9 +59,9 @@ public class TestKeys {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
 
-  private static MiniOzoneCluster cluster = null;
+  private static MiniOzoneCluster ozoneCluster = null;
   static private String path;
-  private static OzoneRestClient client = null;
+  private static OzoneRestClient ozoneRestClient = null;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -82,11 +82,12 @@ public class TestKeys {
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
-    cluster = new MiniOzoneCluster.Builder(conf)
+    ozoneCluster = new MiniOzoneCluster.Builder(conf)
         .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
-    DataNode dataNode = cluster.getDataNodes().get(0);
+    DataNode dataNode = ozoneCluster.getDataNodes().get(0);
     final int port = dataNode.getInfoPort();
-    client = new OzoneRestClient(String.format("http://localhost:%d", port));
+    ozoneRestClient = new OzoneRestClient(
+        String.format("http://localhost:%d", port));
   }
 
   /**
@@ -94,8 +95,8 @@ public class TestKeys {
    */
   @AfterClass
   public static void shutdown() {
-    if (cluster != null) {
-      cluster.shutdown();
+    if (ozoneCluster != null) {
+      ozoneCluster.shutdown();
     }
   }
 
@@ -104,10 +105,10 @@ public class TestKeys {
    *
    * @return File.
    */
-  private File createRandomDataFile(String fileName, long size) {
-    File tmpDir = new File(path);
+  static File createRandomDataFile(String dir, String fileName, long size) {
+    File tmpDir = new File(dir);
     tmpDir.mkdirs();
-    File tmpFile = new File(path + "/" + fileName);
+    File tmpFile = new File(tmpDir, fileName);
     try {
       FileOutputStream randFile = new FileOutputStream(tmpFile);
       Random r = new Random();
@@ -122,12 +123,19 @@ public class TestKeys {
     return tmpFile;
   }
 
+  static class PutHelper {
+    private final OzoneRestClient client;
+    private final String dir;
 
-  private class PutHelper {
     OzoneVolume vol;
     OzoneBucket bucket;
     File file;
 
+    PutHelper(OzoneRestClient client, String dir) {
+      this.client = client;
+      this.dir = dir;
+    }
+
     public OzoneVolume getVol() {
       return vol;
     }
@@ -157,7 +165,7 @@ public class TestKeys {
       bucket = vol.createBucket(bucketName, acls, StorageType.DEFAULT);
 
       String keyName = OzoneUtils.getRequestID().toLowerCase();
-      file = createRandomDataFile(keyName, 1024);
+      file = createRandomDataFile(dir, keyName, 1024);
 
       bucket.putKey(keyName, file);
       return keyName;
@@ -167,7 +175,11 @@ public class TestKeys {
 
   @Test
   public void testPutKey() throws OzoneException {
-    PutHelper helper  = new PutHelper();
+    runTestPutKey(new PutHelper(ozoneRestClient, path));
+  }
+
+  static void runTestPutKey(PutHelper helper) throws OzoneException {
+    final OzoneRestClient client = helper.client;
     helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
@@ -203,7 +215,8 @@ public class TestKeys {
     }
   }
 
-  private void restartDatanode(int datanodeIdx)
+  private static void restartDatanode(
+      MiniOzoneCluster cluster, int datanodeIdx, OzoneRestClient client)
       throws IOException, OzoneException, URISyntaxException {
     cluster.restartDataNode(datanodeIdx);
     // refresh the datanode endpoint uri after datanode restart
@@ -215,17 +228,23 @@ public class TestKeys {
   @Test
   public void testPutAndGetKeyWithDnRestart()
       throws OzoneException, IOException, URISyntaxException {
+    runTestPutAndGetKeyWithDnRestart(
+        new PutHelper(ozoneRestClient, path), ozoneCluster);
+  }
 
-    PutHelper helper  = new PutHelper();
+  static void runTestPutAndGetKeyWithDnRestart(
+      PutHelper helper, MiniOzoneCluster cluster)
+      throws OzoneException, IOException, URISyntaxException {
     String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
     // restart the datanode
-    restartDatanode(0);
+    restartDatanode(cluster, 0, helper.client);
 
     // verify getKey after the datanode restart
-    String newFileName =  path + "/" +OzoneUtils.getRequestID().toLowerCase();
+    String newFileName = helper.dir + "/"
+        + OzoneUtils.getRequestID().toLowerCase();
     Path newPath = Paths.get(newFileName);
 
     helper.getBucket().getKey(keyName, newPath);
@@ -244,14 +263,21 @@ public class TestKeys {
 
   @Test
   public void testPutAndGetKey() throws OzoneException, IOException {
+    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path));
+  }
+
+  static void runTestPutAndGetKey(PutHelper helper)
+      throws OzoneException, IOException {
+    final OzoneRestClient client = helper.client;
 
-    PutHelper helper  = new PutHelper();
     String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
-    String newFileName1 =  path + "/" +OzoneUtils.getRequestID().toLowerCase();
-    String newFileName2 =  path + "/" +OzoneUtils.getRequestID().toLowerCase();
+    final String newFileName1 =  helper.dir + "/"
+        + OzoneUtils.getRequestID().toLowerCase();
+    final String newFileName2 =  helper.dir + "/"
+        + OzoneUtils.getRequestID().toLowerCase();
 
     Path newPath1 = Paths.get(newFileName1);
     Path newPath2 = Paths.get(newFileName2);
@@ -300,8 +326,11 @@ public class TestKeys {
 
   @Test
   public void testPutAndDeleteKey() throws OzoneException, IOException {
+    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path));
+  }
 
-    PutHelper helper  = new PutHelper();
+  static void runTestPutAndDeleteKey(PutHelper helper)
+      throws OzoneException, IOException {
     String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
@@ -318,7 +347,12 @@ public class TestKeys {
 
   @Test
   public void testPutAndListKey() throws OzoneException, IOException {
-    PutHelper helper  = new PutHelper();
+    runTestPutAndListKey(new PutHelper(ozoneRestClient, path));
+  }
+
+  static void runTestPutAndListKey(PutHelper helper)
+      throws OzoneException, IOException {
+    final OzoneRestClient client = helper.client;
     helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
@@ -382,7 +416,10 @@ public class TestKeys {
 
   @Test
   public void testGetKeyInfo() throws OzoneException, IOException {
-    PutHelper helper = new PutHelper();
+    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path));
+  }
+
+  static void runTestGetKeyInfo(PutHelper helper) throws OzoneException {
     String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
new file mode 100644
index 0000000..b0fea90
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
@@ -0,0 +1,88 @@
+/*
+ * 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.hadoop.ozone.web.client;
+
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+import static org.apache.hadoop.ozone.web.client.TestKeys.*;
+
+/** The same as {@link TestKeys} except that this test is Ratis enabled. */
+public class TestKeysRatis {
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static RatisTestHelper.RatisTestSuite suite;
+  private static String path;
+  private static OzoneRestClient ozoneRestClient;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    suite = new RatisTestHelper.RatisTestSuite(TestKeysRatis.class);
+    path = suite.getConf().get(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT);
+    ozoneRestClient = suite.newOzoneRestClient();
+  }
+
+  @AfterClass
+  public static void shutdown() {
+    if (suite != null) {
+      suite.close();
+    }
+  }
+
+  @Test
+  public void testPutKey() throws OzoneException {
+    runTestPutKey(new PutHelper(ozoneRestClient, path));
+  }
+
+  @Test
+  public void testPutAndGetKeyWithDnRestart()
+      throws OzoneException, IOException, URISyntaxException {
+    runTestPutAndGetKeyWithDnRestart(
+        new PutHelper(ozoneRestClient, path), suite.getCluster());
+  }
+
+  @Test
+  public void testPutAndGetKey() throws OzoneException, IOException {
+    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path));
+  }
+
+  @Test
+  public void testPutAndDeleteKey() throws OzoneException, IOException {
+    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path));
+  }
+
+  @Test
+  public void testPutAndListKey() throws OzoneException, IOException {
+    runTestPutAndListKey(new PutHelper(ozoneRestClient, path));
+  }
+
+  @Test
+  public void testGetKeyInfo() throws OzoneException, IOException {
+    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
index 21d3b2a..3762bd5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -36,10 +36,7 @@ import org.apache.http.client.methods.HttpUriRequest;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.*;
 import org.mockito.Mockito;
 
 import java.io.File;
@@ -58,7 +55,7 @@ import static org.mockito.Mockito.verify;
 
 public class TestVolume {
   private static MiniOzoneCluster cluster = null;
-  private static OzoneRestClient client = null;
+  private static OzoneRestClient ozoneRestClient = null;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -87,7 +84,8 @@ public class TestVolume {
     DataNode dataNode = cluster.getDataNodes().get(0);
     final int port = dataNode.getInfoPort();
 
-    client = new OzoneRestClient(String.format("http://localhost:%d", port));
+    ozoneRestClient = new OzoneRestClient(
+        String.format("http://localhost:%d", port));
   }
 
   /**
@@ -102,6 +100,11 @@ public class TestVolume {
 
   @Test
   public void testCreateVolume() throws OzoneException, IOException {
+    runTestCreateVolume(ozoneRestClient);
+  }
+
+  static void runTestCreateVolume(OzoneRestClient client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
 
@@ -120,6 +123,11 @@ public class TestVolume {
 
   @Test
   public void testCreateDuplicateVolume() throws OzoneException {
+    runTestCreateDuplicateVolume(ozoneRestClient);
+  }
+
+  static void runTestCreateDuplicateVolume(OzoneRestClient client)
+      throws OzoneException {
     try {
       client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
       client.createVolume("testvol", "bilbo", "100TB");
@@ -134,6 +142,11 @@ public class TestVolume {
 
   @Test
   public void testDeleteVolume() throws OzoneException {
+    runTestDeleteVolume(ozoneRestClient);
+  }
+
+  static void runTestDeleteVolume(OzoneRestClient client)
+      throws OzoneException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
@@ -142,6 +155,11 @@ public class TestVolume {
 
   @Test
   public void testChangeOwnerOnVolume() throws OzoneException {
+    runTestChangeOwnerOnVolume(ozoneRestClient);
+  }
+
+  static void runTestChangeOwnerOnVolume(OzoneRestClient client)
+      throws OzoneException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
@@ -152,6 +170,11 @@ public class TestVolume {
 
   @Test
   public void testChangeQuotaOnVolume() throws OzoneException, IOException {
+    runTestChangeQuotaOnVolume(ozoneRestClient);
+  }
+
+  static void runTestChangeQuotaOnVolume(OzoneRestClient client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
@@ -163,6 +186,11 @@ public class TestVolume {
 
   @Test
   public void testListVolume() throws OzoneException, IOException {
+    runTestListVolume(ozoneRestClient);
+  }
+
+  static void runTestListVolume(OzoneRestClient client)
+      throws OzoneException, IOException {
     client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     for (int x = 0; x < 10; x++) {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
@@ -174,9 +202,15 @@ public class TestVolume {
     assertTrue(ovols.size() >= 10);
   }
 
-  //@Test
-  // Takes 3m to run, disable for now.
+  // TODO: remove @Ignore below once the problem has been resolved.
+  @Ignore("Takes 3m to run, disable for now.")
+  @Test
   public void testListVolumePagination() throws OzoneException, IOException {
+    runTestListVolumePagination(ozoneRestClient);
+  }
+
+  static void runTestListVolumePagination(OzoneRestClient client)
+      throws OzoneException, IOException {
     final int volCount = 2000;
     final int step = 100;
     client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
@@ -198,8 +232,15 @@ public class TestVolume {
     Assert.assertEquals(volCount / step, pagecount);
   }
 
-  //@Test
+  // TODO: remove @Ignore below once the problem has been resolved.
+  @Ignore
+  @Test
   public void testListAllVolumes() throws OzoneException, IOException {
+    runTestListAllVolumes(ozoneRestClient);
+  }
+
+  static void runTestListAllVolumes(OzoneRestClient client)
+      throws OzoneException, IOException {
     final int volCount = 200;
     final int step = 10;
     client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
@@ -230,6 +271,11 @@ public class TestVolume {
 
   @Test
   public void testListVolumes() throws OzoneException, IOException {
+    runTestListVolumes(ozoneRestClient);
+  }
+
+  static void runTestListVolumes(OzoneRestClient client)
+      throws OzoneException, IOException {
     final int volCount = 20;
     final String user1 = "test-user-a";
     final String user2 = "test-user-b";
@@ -288,12 +334,12 @@ public class TestVolume {
    * of this method is always used as the input of
    * {@link TestVolume#verifyHttpConnectionClosed(List)}.
    *
-   * @param ozoneRestClient mocked ozone client.
+   * @param mockedClient mocked ozone client.
    * @return a list of mocked {@link CloseableHttpClient}.
    * @throws IOException
    */
-  private List<CloseableHttpClient> mockHttpClients(
-      OzoneRestClient ozoneRestClient)
+  private static List<CloseableHttpClient> mockHttpClients(
+      OzoneRestClient mockedClient)
       throws IOException {
     List<CloseableHttpClient> spyHttpClients = new ArrayList<>();
     for (int i = 0; i < 5; i++) {
@@ -304,7 +350,7 @@ public class TestVolume {
 
     List<CloseableHttpClient> nextReturns =
         new ArrayList<>(spyHttpClients.subList(1, spyHttpClients.size()));
-    Mockito.when(ozoneRestClient.newHttpClient()).thenReturn(
+    Mockito.when(mockedClient.newHttpClient()).thenReturn(
         spyHttpClients.get(0),
         nextReturns.toArray(new CloseableHttpClient[nextReturns.size()]));
     return spyHttpClients;
@@ -320,7 +366,7 @@ public class TestVolume {
    *
    * @param mockedHttpClients
    */
-  private void verifyHttpConnectionClosed(
+  private static void verifyHttpConnectionClosed(
       List<CloseableHttpClient> mockedHttpClients) {
     final AtomicInteger totalCalled = new AtomicInteger();
     Assert.assertTrue(mockedHttpClients.stream().allMatch(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
new file mode 100644
index 0000000..91e6d2d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
@@ -0,0 +1,99 @@
+/*
+ * 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.hadoop.ozone.web.client;
+
+import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.junit.*;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+
+/** The same as {@link TestVolume} except that this test is Ratis enabled. */
+public class TestVolumeRatis {
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static RatisTestHelper.RatisTestSuite suite;
+  private static OzoneRestClient ozoneClient;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    suite = new RatisTestHelper.RatisTestSuite(TestVolumeRatis.class);
+    ozoneClient = suite.newOzoneRestClient();
+  }
+
+  @AfterClass
+  public static void shutdown() {
+    if (suite != null) {
+      suite.close();
+    }
+  }
+
+  @Test
+  public void testCreateVolume() throws OzoneException, IOException {
+    TestVolume.runTestCreateVolume(ozoneClient);
+  }
+
+  @Test
+  public void testCreateDuplicateVolume() throws OzoneException {
+    TestVolume.runTestCreateDuplicateVolume(ozoneClient);
+  }
+
+  @Test
+  public void testDeleteVolume() throws OzoneException {
+    TestVolume.runTestDeleteVolume(ozoneClient);
+  }
+
+  @Test
+  public void testChangeOwnerOnVolume() throws OzoneException {
+    TestVolume.runTestChangeOwnerOnVolume(ozoneClient);
+  }
+
+  @Test
+  public void testChangeQuotaOnVolume() throws OzoneException, IOException {
+    TestVolume.runTestChangeQuotaOnVolume(ozoneClient);
+  }
+
+  // TODO: remove @Ignore below once the problem has been resolved.
+  @Ignore("listVolumes not implemented in DistributedStorageHandler")
+  @Test
+  public void testListVolume() throws OzoneException, IOException {
+    TestVolume.runTestListVolume(ozoneClient);
+  }
+
+  // TODO: remove @Ignore below once the problem has been resolved.
+  @Ignore("See TestVolume.testListVolumePagination()")
+  @Test
+  public void testListVolumePagination() throws OzoneException, IOException {
+    TestVolume.runTestListVolumePagination(ozoneClient);
+  }
+
+  // TODO: remove @Ignore below once the problem has been resolved.
+  @Ignore("See TestVolume.testListAllVolumes()")
+  @Test
+  public void testListAllVolumes() throws OzoneException, IOException {
+    TestVolume.runTestListAllVolumes(ozoneClient);
+  }
+
+  @Test
+  public void testListVolumes() throws OzoneException, IOException {
+    TestVolume.runTestListVolumes(ozoneClient);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90f1d585/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
index 47bbd4e..2d1c98b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
@@ -60,7 +60,7 @@ log4j.additivity.org.apache.hadoop.ozone=false
 log4j.appender.OZONE=org.apache.log4j.ConsoleAppender
 log4j.appender.OZONE.Threshold=ALL
 log4j.appender.OZONE.layout=org.apache.log4j.PatternLayout
-log4j.appender.OZONE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p \
+log4j.appender.OZONE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) \
  %X{component} %X{function} %X{resource} %X{user} %X{request} - %m%n
 
 # Real ozone logger that writes to ozone.log


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org