Ver Fonte

HDFS-10420. Fix Ozone unit tests to use MiniOzoneCluster. Contributed by Arpit Agarwal.

Arpit Agarwal há 9 anos atrás
pai
commit
54f7975dfc
16 ficheiros alterados com 200 adições e 159 exclusões
  1. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientUtils.java
  2. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerLocationManagerImpl.java
  3. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
  4. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
  5. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
  6. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/storage/StorageContainerManager.java
  7. 42 18
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
  8. 21 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
  9. 22 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
  10. 16 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
  11. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
  12. 14 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
  13. 23 25
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
  14. 12 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
  15. 14 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
  16. 6 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientUtils.java

@@ -110,8 +110,10 @@ public final class OzoneClientUtils {
     final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
         OZONE_SCM_DATANODE_ADDRESS_KEY);
 
-    return NetUtils.createSocketAddr(host.get() + ":" +
+    InetSocketAddress addr = NetUtils.createSocketAddr(host.get() + ":" +
         port.or(OZONE_SCM_DATANODE_PORT_DEFAULT));
+
+    return addr;
   }
 
   /**

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerLocationManagerImpl.java

@@ -49,7 +49,7 @@ public class ContainerLocationManagerImpl implements ContainerLocationManager {
 
 
   private final Configuration conf;
-  private final FsDatasetSpi dataset;
+  private final FsDatasetSpi<? extends FsVolumeSpi> dataset;
   private final Path[] volumePaths;
   private int currentIndex;
   private final List<Path> locations;
@@ -59,8 +59,9 @@ public class ContainerLocationManagerImpl implements ContainerLocationManager {
    * Constructs a Location Manager.
    * @param conf - Configuration.
    */
-  public ContainerLocationManagerImpl(Configuration conf, List<Path> locations,
-                                      FsDatasetSpi dataset) throws IOException {
+  public ContainerLocationManagerImpl(
+      Configuration conf, List<Path> locations,
+      FsDatasetSpi<? extends FsVolumeSpi> dataset) throws IOException {
     this.conf = conf;
     this.dataset = dataset;
     List<Path> pathList = new LinkedList<>();

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
@@ -83,9 +84,9 @@ public class ContainerManagerImpl implements ContainerManager {
    * @throws IOException
    */
   @Override
-  public void init(Configuration config, List<Path> containerDirs,
-                   FsDatasetSpi dataset)
-      throws IOException {
+  public void init(
+      Configuration config, List<Path> containerDirs,
+      FsDatasetSpi<? extends FsVolumeSpi> dataset) throws IOException {
 
     Preconditions.checkNotNull(config);
     Preconditions.checkNotNull(containerDirs);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.Pipeline;
@@ -42,11 +43,10 @@ public interface ContainerManager extends RwLock {
    *
    * @param config        - Configuration.
    * @param containerDirs - List of Metadata Container locations.
-   * @param dataset       - FSDataset.
    * @throws IOException
    */
   void init(Configuration config, List<Path> containerDirs,
-            FsDatasetSpi dataset)
+            FsDatasetSpi<? extends FsVolumeSpi> dataset)
       throws IOException;
 
   /**

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java

@@ -48,7 +48,7 @@ public class OzoneContainer {
       LoggerFactory.getLogger(OzoneContainer.class);
 
   private final Configuration ozoneConfig;
-  private final FsDatasetSpi dataSet;
+  private final FsDatasetSpi<? extends FsVolumeSpi> dataSet;
   private final ContainerDispatcher dispatcher;
   private final ContainerManager manager;
   private final XceiverServer server;
@@ -62,8 +62,9 @@ public class OzoneContainer {
    * @param dataSet     - FsDataset.
    * @throws IOException
    */
-  public OzoneContainer(Configuration ozoneConfig, FsDatasetSpi dataSet) throws
-      Exception {
+  public OzoneContainer(
+      Configuration ozoneConfig,
+      FsDatasetSpi<? extends FsVolumeSpi> dataSet) throws Exception {
     List<Path> locations = new LinkedList<>();
     String[] paths = ozoneConfig.getStrings(OzoneConfigKeys
         .OZONE_METADATA_DIRS);
@@ -150,8 +151,9 @@ public class OzoneContainer {
    * @param pathList - List of paths.
    * @throws IOException
    */
-  private void getDataDir(FsDatasetSpi dataset, List<Path> pathList) throws
-      IOException {
+  private void getDataDir(
+      FsDatasetSpi<? extends FsVolumeSpi> dataset,
+      List<Path> pathList) throws IOException {
     FsDatasetSpi.FsVolumeReferences references;
     try {
       synchronized (dataset) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/storage/StorageContainerManager.java

@@ -155,7 +155,7 @@ public class StorageContainerManager
         DatanodeProtocolPB.class, dnProtoPbService, handlerCount);
     datanodeRpcAddress = updateListenAddress(conf,
         OZONE_SCM_DATANODE_ADDRESS_KEY, datanodeRpcAddr, datanodeRpcServer);
-    LOG.info(buildRpcServerStartMessage("Service RPC server",
+    LOG.info(buildRpcServerStartMessage("RPC server for DataNodes",
         datanodeRpcAddress));
 
     BlockingService storageProtoPbService =
@@ -522,7 +522,7 @@ public class StorageContainerManager
     InetSocketAddress updatedAddr = new InetSocketAddress(
         addr.getHostString(), listenAddr.getPort());
     conf.set(rpcAddressKey,
-        addr.getHostString() + ":" + listenAddr.getPort());
+        addr.getHostString() + ":" + updatedAddr.getPort());
     return updatedAddr;
   }
 

+ 42 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java

@@ -22,9 +22,14 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URISyntaxException;
 import java.util.Random;
+import java.util.concurrent.TimeoutException;
 
+import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 
+import com.google.common.base.Supplier;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,7 +46,6 @@ import org.apache.hadoop.ozone.storage.StorageContainerManager;
 import org.apache.hadoop.ozone.web.client.OzoneClient;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Time;
 
 /**
  * MiniOzoneCluster creates a complete in-process Ozone cluster suitable for
@@ -82,6 +86,7 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
       extends org.apache.hadoop.hdfs.MiniDFSCluster.Builder {
 
     private final OzoneConfiguration conf;
+    private Optional<String> ozoneHandlerType = Optional.absent();
 
     /**
      * Creates a new Builder.
@@ -100,13 +105,35 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
       return this;
     }
 
+    public Builder setHandlerType(String handler) {
+      ozoneHandlerType = Optional.of(handler);
+      return this;
+    }
+
     @Override
     public MiniOzoneCluster build() throws IOException {
+      if (!ozoneHandlerType.isPresent()) {
+        throw new IllegalArgumentException(
+            "The Ozone handler type must be specified.");
+      }
+
+      conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
+      conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY, true);
+      conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, ozoneHandlerType.get());
       conf.set(OzoneConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
       conf.set(OzoneConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
       StorageContainerManager scm = new StorageContainerManager(conf);
       scm.start();
-      return new MiniOzoneCluster(this, scm);
+      MiniOzoneCluster cluster = new MiniOzoneCluster(this, scm);
+      try {
+        cluster.waitOzoneReady();
+      } catch(Exception e) {
+        // A workaround to propagate MiniOzoneCluster failures without
+        // changing the method signature (which would require cascading
+        // changes to hundreds of unrelated HDFS tests).
+        throw new IOException("Failed to start MiniOzoneCluster", e);
+      }
+      return cluster;
     }
   }
 
@@ -176,22 +203,19 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
   /**
    * Waits for the Ozone cluster to be ready for processing requests.
    */
-  public void waitOzoneReady() {
-    long begin = Time.monotonicNow();
-    while (scm.getDatanodeReport(DatanodeReportType.LIVE).length <
-        numDataNodes) {
-      if (Time.monotonicNow() - begin > 20000) {
-        throw new IllegalStateException(
-            "Timed out waiting for Ozone cluster to become ready.");
+  public void waitOzoneReady() throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        final DatanodeInfo[] reports =
+            scm.getDatanodeReport(DatanodeReportType.LIVE);
+        if (reports.length >= numDataNodes) {
+          return true;
+        }
+        LOG.info("Waiting for cluster to be ready. Got {} of {} DN reports.",
+            reports.length, numDataNodes);
+        return false;
       }
-      LOG.info("Waiting for Ozone cluster to become ready");
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new IllegalStateException(
-            "Interrupted while waiting for Ozone cluster to become ready.");
-      }
-    }
+    }, 100, 45000);
   }
 }

+ 21 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/storage/TestStorageContainerManager.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java

@@ -33,8 +33,17 @@ import org.junit.rules.ExpectedException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.protocol.LocatedContainer;
 import org.apache.hadoop.ozone.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.junit.rules.Timeout;
 
+/**
+ * Test class that exercises the StorageContainerManager.
+ */
 public class TestStorageContainerManager {
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
 
   private static MiniOzoneCluster cluster;
   private static OzoneConfiguration conf;
@@ -47,9 +56,6 @@ public class TestStorageContainerManager {
   @BeforeClass
   public static void init() throws IOException {
     conf = new OzoneConfiguration();
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "distributed");
-    conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY, true);
   }
 
   @After
@@ -58,9 +64,9 @@ public class TestStorageContainerManager {
   }
 
   @Test
-  public void testLocationsForSingleKey() throws IOException {
-    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitOzoneReady();
+  public void testLocationsForSingleKey() throws Exception {
+    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(1)
+        .setHandlerType("distributed").build();
     storageContainerLocationClient =
         cluster.createStorageContainerLocationClient();
     Set<LocatedContainer> containers =
@@ -72,9 +78,9 @@ public class TestStorageContainerManager {
   }
 
   @Test
-  public void testLocationsForMultipleKeys() throws IOException {
-    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitOzoneReady();
+  public void testLocationsForMultipleKeys() throws Exception {
+    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(1)
+        .setHandlerType("distributed").build();
     storageContainerLocationClient =
         cluster.createStorageContainerLocationClient();
     Set<LocatedContainer> containers =
@@ -88,9 +94,9 @@ public class TestStorageContainerManager {
   }
 
   @Test
-  public void testNoDataNodes() throws IOException {
-    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(0).build();
-    cluster.waitOzoneReady();
+  public void testNoDataNodes() throws Exception {
+    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(0)
+        .setHandlerType("distributed").build();
     storageContainerLocationClient =
         cluster.createStorageContainerLocationClient();
     exception.expect(IOException.class);
@@ -100,9 +106,9 @@ public class TestStorageContainerManager {
   }
 
   @Test
-  public void testMultipleDataNodes() throws IOException {
-    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(3).build();
-    cluster.waitOzoneReady();
+  public void testMultipleDataNodes() throws Exception {
+    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(3)
+        .setHandlerType("distributed").build();
     storageContainerLocationClient =
         cluster.createStorageContainerLocationClient();
     Set<LocatedContainer> containers =

+ 22 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java

@@ -20,9 +20,10 @@ package org.apache.hadoop.ozone.container.common.impl;
 
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -41,6 +42,7 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
+import org.junit.rules.Timeout;
 
 import java.io.File;
 import java.io.IOException;
@@ -69,21 +71,26 @@ import static org.junit.Assert.fail;
  * Simple tests to verify that container persistence works as expected.
  */
 public class TestContainerPersistence {
-
-  static String path;
-  static ContainerManagerImpl containerManager;
-  static ChunkManagerImpl chunkManager;
-  static KeyManagerImpl keyManager;
-  static OzoneConfiguration conf;
-  static FsDatasetSpi fsDataSet;
-  static MiniDFSCluster cluster;
-  static List<Path> pathLists = new LinkedList<>();
-
   @Rule
   public ExpectedException exception = ExpectedException.none();
 
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static String path;
+  private static ContainerManagerImpl containerManager;
+  private static ChunkManagerImpl chunkManager;
+  private static KeyManagerImpl keyManager;
+  private static OzoneConfiguration conf;
+  private static FsDatasetSpi<? extends FsVolumeSpi> fsDataSet;
+  private static MiniOzoneCluster cluster;
+  private static List<Path> pathLists = new LinkedList<>();
+
   @BeforeClass
-  public static void init() throws IOException {
+  public static void init() throws Throwable {
     conf = new OzoneConfiguration();
     URL p = conf.getClass().getResource("");
     path = p.getPath().concat(
@@ -91,8 +98,6 @@ public class TestContainerPersistence {
     path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
         OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
 
     File containerDir = new File(path);
     if (containerDir.exists()) {
@@ -101,8 +106,8 @@ public class TestContainerPersistence {
 
     Assert.assertTrue(containerDir.mkdirs());
 
-    cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
+    cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType("local").build();
     fsDataSet = cluster.getDataNodes().get(0).getFSDataset();
     containerManager = new ContainerManagerImpl();
     chunkManager = new ChunkManagerImpl(containerManager);
@@ -601,4 +606,4 @@ public class TestContainerPersistence {
   }
 
 
-}
+}

+ 16 - 16
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java

@@ -17,10 +17,8 @@
  */
 
 package org.apache.hadoop.ozone.container.ozoneimpl;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
@@ -28,15 +26,23 @@ import org.apache.hadoop.ozone.container.common.helpers.Pipeline;
 import org.apache.hadoop.ozone.container.common.transport.client.XceiverClient;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import java.net.URL;
 
 public class TestOzoneContainer {
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
   @Test
   public void testCreateOzoneContainer() throws Exception {
     String containerName = OzoneUtils.getRequestID();
-    Configuration conf = new OzoneConfiguration();
+    OzoneConfiguration conf = new OzoneConfiguration();
     URL p = conf.getClass().getResource("");
     String path = p.getPath().concat(
         TestOzoneContainer.class.getSimpleName());
@@ -44,15 +50,11 @@ public class TestOzoneContainer {
         OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
 
+    MiniOzoneCluster cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType("local").build();
+
     // We don't start Ozone Container via data node, we will do it
     // independently in our test path.
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, false);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
-
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
-
-
     Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline
         (containerName);
     conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
@@ -78,7 +80,7 @@ public class TestOzoneContainer {
   public void testOzoneContainerViaDataNode() throws Exception {
     String keyName = OzoneUtils.getRequestID();
     String containerName = OzoneUtils.getRequestID();
-    Configuration conf = new OzoneConfiguration();
+    OzoneConfiguration conf = new OzoneConfiguration();
     URL p = conf.getClass().getResource("");
     String path = p.getPath().concat(
         TestOzoneContainer.class.getSimpleName());
@@ -87,16 +89,14 @@ public class TestOzoneContainer {
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
 
     // Start ozone container Via Datanode create.
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
 
     Pipeline pipeline =
         ContainerTestHelper.createSingleNodePipeline(containerName);
     conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
         pipeline.getLeader().getContainerPort());
 
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
+    MiniOzoneCluster cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType("local").build();
 
     // This client talks to ozone container via datanode.
     XceiverClient client = new XceiverClient(pipeline, conf);

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java

@@ -31,17 +31,22 @@ import org.junit.rules.ExpectedException;
 
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.client.OzoneBucket;
 import org.apache.hadoop.ozone.web.client.OzoneClient;
 import org.apache.hadoop.ozone.web.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
+import org.junit.rules.Timeout;
 
 /**
  * End-to-end testing of Ozone REST operations.
  */
 public class TestOzoneRestWithMiniCluster {
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
 
   private static MiniOzoneCluster cluster;
   private static OzoneConfiguration conf;
@@ -54,11 +59,8 @@ public class TestOzoneRestWithMiniCluster {
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "distributed");
-    conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY, true);
-    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(3).build();
-    cluster.waitOzoneReady();
+    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(3)
+        .setHandlerType("distributed").build();
     ozoneClient = cluster.createOzoneClient();
   }
 

+ 14 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java

@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.ozone.web;
 
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
@@ -35,7 +35,9 @@ import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import javax.ws.rs.core.HttpHeaders;
 import java.io.IOException;
@@ -49,8 +51,14 @@ import static java.net.HttpURLConnection.HTTP_OK;
 import static org.junit.Assert.assertEquals;
 
 public class TestOzoneVolumes {
-  static MiniDFSCluster cluster = null;
-  static int port = 0;
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static MiniOzoneCluster cluster = null;
+  private static int port = 0;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -62,7 +70,7 @@ public class TestOzoneVolumes {
    * @throws IOException
    */
   @BeforeClass
-  public static void init() throws IOException {
+  public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
 
     URL p = conf.getClass().getResource("");
@@ -71,13 +79,10 @@ public class TestOzoneVolumes {
         OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
 
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
-    conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY, true);
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
-    cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
+    cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType("local").build();
     DataNode dataNode = cluster.getDataNodes().get(0);
     port = dataNode.getInfoPort();
   }

+ 23 - 25
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java

@@ -18,43 +18,43 @@
 
 package org.apache.hadoop.ozone.web;
 
-import static java.net.HttpURLConnection.HTTP_CREATED;
-import static org.apache.hadoop.ozone.web.utils.OzoneUtils.*;
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.net.URL;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import javax.ws.rs.core.HttpHeaders;
-
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
-import org.apache.hadoop.ozone.web.headers.Header;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.web.headers.Header;
 import org.apache.hadoop.util.Time;
-
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.impl.client.DefaultHttpClient;
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 
-public class TestOzoneWebAccess {
+import javax.ws.rs.core.HttpHeaders;
+import java.io.IOException;
+import java.net.URL;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Locale;
 
-  private static MiniDFSCluster cluster;
-  private static int port;
+import static java.net.HttpURLConnection.HTTP_CREATED;
+import static org.apache.hadoop.ozone.web.utils.OzoneUtils.getRequestID;
+import static org.junit.Assert.assertEquals;
 
+public class TestOzoneWebAccess {
+  /**
+   * Set the timeout for every test
+   */
   @Rule
-  public Timeout timeout = new Timeout(30000);
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static MiniOzoneCluster cluster;
+  private static int port;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -66,23 +66,21 @@ public class TestOzoneWebAccess {
    * @throws IOException
    */
   @BeforeClass
-  public static void init() throws IOException {
+  public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
 
     URL p = conf.getClass().getResource("");
     String path = p.getPath().concat(TestOzoneWebAccess.class.getSimpleName());
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
 
-    cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
+    cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType("local").build();
     DataNode dataNode = cluster.getDataNodes().get(0);
     port = dataNode.getInfoPort();
   }
 
   /**
-   * shutdown MiniDFSCluster
+   * shutdown MiniOzoneCluster.
    */
   @AfterClass
   public static void shutdown() {

+ 12 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java

@@ -18,8 +18,8 @@
 package org.apache.hadoop.ozone.web.client;
 
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
@@ -27,7 +27,9 @@ import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 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;
@@ -39,9 +41,13 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 public class TestBuckets {
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
 
-  static MiniDFSCluster cluster = null;
-  static int port = 0;
+  private static MiniOzoneCluster cluster = null;
   private static OzoneClient client = null;
 
   /**
@@ -64,13 +70,10 @@ public class TestBuckets {
         OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
 
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
-
-    cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
+    cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType("local").build();
     DataNode dataNode = cluster.getDataNodes().get(0);
-    port = dataNode.getInfoPort();
+    final int port = dataNode.getInfoPort();
     client = new OzoneClient(String.format("http://localhost:%d", port));
   }
 

+ 14 - 16
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java

@@ -19,8 +19,8 @@ package org.apache.hadoop.ozone.web.client;
 
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
@@ -31,13 +31,14 @@ import org.apache.log4j.Logger;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -49,12 +50,16 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 public class TestKeys {
-  static MiniDFSCluster cluster = null;
-  static int port = 0;
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static MiniOzoneCluster cluster = null;
   static private String path;
   private static OzoneClient client = null;
 
-
   /**
    * Create a MiniDFSCluster for testing.
    *
@@ -65,27 +70,20 @@ public class TestKeys {
    * @throws IOException
    */
   @BeforeClass
-  public static void init()
-      throws IOException, OzoneException, URISyntaxException {
+  public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
 
     URL p = conf.getClass().getResource("");
     path = p.getPath().concat(TestKeys.class.getSimpleName());
     path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
                             OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
-
-    conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY, true);
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
-
-    cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
+    cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType("local").build();
     DataNode dataNode = cluster.getDataNodes().get(0);
-    port = dataNode.getInfoPort();
+    final int port = dataNode.getInfoPort();
     client = new OzoneClient(String.format("http://localhost:%d", port));
   }
 

+ 6 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java

@@ -19,8 +19,8 @@
 package org.apache.hadoop.ozone.web.client;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
@@ -36,7 +36,6 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.List;
 
@@ -46,8 +45,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 public class TestVolume {
-  private static MiniDFSCluster cluster = null;
-  private static int port = 0;
+  private static MiniOzoneCluster cluster = null;
   private static OzoneClient client = null;
 
   /**
@@ -60,8 +58,7 @@ public class TestVolume {
    * @throws IOException
    */
   @BeforeClass
-  public static void init() throws IOException, OzoneException,
-      URISyntaxException {
+  public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
 
     URL p = conf.getClass().getResource("");
@@ -71,15 +68,12 @@ public class TestVolume {
     FileUtils.deleteDirectory(new File(path));
 
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
-    conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY, true);
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
-    cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
+    cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType("local").build();
     DataNode dataNode = cluster.getDataNodes().get(0);
-    port = dataNode.getInfoPort();
+    final int port = dataNode.getInfoPort();
 
     client = new OzoneClient(String.format("http://localhost:%d", port));
   }