瀏覽代碼

HDDS-111. Include tests for Rest Client in TestVolume and TestBucket.
Contributed by Lokesh Jain.

Anu Engineer 7 年之前
父節點
當前提交
5c7ad52573

+ 9 - 9
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java

@@ -18,14 +18,12 @@
 package org.apache.hadoop.ozone.client;
 
 import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
 import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
 import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
 import org.apache.hadoop.ozone.client.rest.response.VolumeOwner;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-
-import static org.apache.hadoop.ozone.web.utils.OzoneUtils.formatTime;
 
 /** A utility class for OzoneClient. */
 public final class OzoneClientUtils {
@@ -43,7 +41,8 @@ public final class OzoneClientUtils {
   public static BucketInfo asBucketInfo(OzoneBucket bucket) {
     BucketInfo bucketInfo =
         new BucketInfo(bucket.getVolumeName(), bucket.getName());
-    bucketInfo.setCreatedOn(OzoneUtils.formatTime(bucket.getCreationTime()));
+    bucketInfo
+        .setCreatedOn(HddsClientUtils.formatDateTime(bucket.getCreationTime()));
     bucketInfo.setStorageType(bucket.getStorageType());
     bucketInfo.setVersioning(
         OzoneConsts.Versioning.getVersioning(bucket.getVersioning()));
@@ -60,9 +59,9 @@ public final class OzoneClientUtils {
    * @return VolumeInfo instance
    */
   public static VolumeInfo asVolumeInfo(OzoneVolume volume) {
-    VolumeInfo volumeInfo =
-        new VolumeInfo(volume.getName(), formatTime(volume.getCreationTime()),
-            volume.getOwner());
+    VolumeInfo volumeInfo = new VolumeInfo(volume.getName(),
+        HddsClientUtils.formatDateTime(volume.getCreationTime()),
+        volume.getOwner());
     volumeInfo.setQuota(OzoneQuota.getOzoneQuota(volume.getQuota()));
     volumeInfo.setOwner(new VolumeOwner(volume.getOwner()));
     return volumeInfo;
@@ -79,8 +78,9 @@ public final class OzoneClientUtils {
   public static KeyInfo asKeyInfo(OzoneKey key) {
     KeyInfo keyInfo = new KeyInfo();
     keyInfo.setKeyName(key.getName());
-    keyInfo.setCreatedOn(formatTime(key.getCreationTime()));
-    keyInfo.setModifiedOn(formatTime(key.getModificationTime()));
+    keyInfo.setCreatedOn(HddsClientUtils.formatDateTime(key.getCreationTime()));
+    keyInfo.setModifiedOn(
+        HddsClientUtils.formatDateTime(key.getModificationTime()));
     keyInfo.setSize(key.getDataSize());
     return keyInfo;
   }

+ 6 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java

@@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory;
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.URISyntaxException;
+import java.util.concurrent.TimeoutException;
 
 /**
  * Helpers for Ratis tests.
@@ -54,7 +55,8 @@ public interface RatisTestHelper {
      *   RATIS_ENABLED = true, and
      *   OZONE_HANDLER_TYPE_KEY = "distributed".
      */
-    public RatisTestSuite(final Class<?> clazz) throws IOException {
+    public RatisTestSuite(final Class<?> clazz)
+        throws IOException, TimeoutException, InterruptedException {
       conf = newOzoneConfiguration(clazz, RPC);
       cluster = newMiniOzoneCluster(NUM_DATANODES, conf);
     }
@@ -99,9 +101,11 @@ public interface RatisTestHelper {
   }
 
   static MiniOzoneCluster newMiniOzoneCluster(
-      int numDatanodes, OzoneConfiguration conf) throws IOException {
+      int numDatanodes, OzoneConfiguration conf)
+      throws IOException, TimeoutException, InterruptedException {
     final MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(numDatanodes).build();
+    cluster.waitForClusterToBeReady();
     return cluster;
   }
 }

+ 7 - 4
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.ksm;
 
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.net.NetUtils;
@@ -1115,10 +1116,12 @@ public class TestKeySpaceManager {
     KeyInfo keyInfo = storageHandler.getKeyInfo(keyArgs);
     // Compare the time in second unit since the date string reparsed to
     // millisecond will lose precision.
-    Assert.assertTrue((OzoneUtils.formatDate(keyInfo.getCreatedOn())
-        / 1000) >= (currentTime / 1000));
-    Assert.assertTrue((OzoneUtils.formatDate(keyInfo.getModifiedOn())
-        / 1000) >= (currentTime / 1000));
+    Assert.assertTrue(
+        (HddsClientUtils.formatDateTime(keyInfo.getCreatedOn()) / 1000) >= (
+            currentTime / 1000));
+    Assert.assertTrue(
+        (HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()) / 1000) >= (
+            currentTime / 1000));
     Assert.assertEquals(keyName, keyInfo.getKeyName());
     // with out data written, the size would be 0
     Assert.assertEquals(0, keyInfo.getSize());

+ 36 - 4
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java

@@ -27,23 +27,29 @@ import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.rest.RestClient;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
 import java.text.ParseException;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 
 import static org.junit.Assert.assertEquals;
@@ -54,6 +60,7 @@ import static org.junit.Assert.fail;
 /**
  * Test Ozone Bucket Lifecycle.
  */
+@RunWith(value = Parameterized.class)
 public class TestBuckets {
   /**
    * Set the timeout for every test.
@@ -63,6 +70,18 @@ public class TestBuckets {
 
   private static MiniOzoneCluster cluster = null;
   private static ClientProtocol client = null;
+  private static OzoneConfiguration conf;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> clientProtocol() {
+    Object[][] params = new Object[][] {
+        {RpcClient.class},
+        {RestClient.class}};
+    return Arrays.asList(params);
+  }
+
+  @Parameterized.Parameter
+  public static Class clientProtocol;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -74,9 +93,10 @@ public class TestBuckets {
    * @throws IOException
    */
   @BeforeClass
-  public static void init() throws IOException,
-      URISyntaxException, OzoneException {
-    OzoneConfiguration conf = new OzoneConfiguration();
+  public static void init()
+      throws IOException, URISyntaxException, OzoneException, TimeoutException,
+      InterruptedException {
+    conf = new OzoneConfiguration();
 
     String path = GenericTestUtils
         .getTempPath(TestBuckets.class.getSimpleName());
@@ -87,7 +107,16 @@ public class TestBuckets {
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
         .build();
-    client = new RpcClient(conf);
+    cluster.waitForClusterToBeReady();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    if (clientProtocol.equals(RestClient.class)) {
+      client = new RestClient(conf);
+    } else {
+      client = new RpcClient(conf);
+    }
   }
 
   /**
@@ -260,6 +289,9 @@ public class TestBuckets {
 
   @Test
   public void testListBucket() throws Exception {
+    if (clientProtocol.equals(RestClient.class)) {
+      return;
+    }
     runTestListBucket(client);
   }
 

+ 34 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java

@@ -17,31 +17,61 @@
  */
 package org.apache.hadoop.ozone.web.client;
 
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.RatisTestHelper;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.rest.RestClient;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
 
 /** The same as {@link TestBuckets} except that this test is Ratis enabled. */
 @Ignore("Disabling Ratis tests for pipeline work.")
+@RunWith(value = Parameterized.class)
 public class TestBucketsRatis {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
 
   private static RatisTestHelper.RatisTestSuite suite;
   private static ClientProtocol client;
+  private static OzoneConfiguration conf;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> clientProtocol() {
+    Object[][] params = new Object[][] {
+        {RpcClient.class},
+        {RestClient.class}};
+    return Arrays.asList(params);
+  }
+
+  @Parameterized.Parameter
+  public static Class clientProtocol;
 
   @BeforeClass
   public static void init() throws Exception {
     suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
-    client = suite.newOzoneClient();
+    conf = suite.getConf();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    if (clientProtocol.equals(RestClient.class)) {
+      client = new RestClient(conf);
+    } else {
+      client = new RpcClient(conf);
+    }
   }
 
   @AfterClass
@@ -72,6 +102,9 @@ public class TestBucketsRatis {
   }
   @Test
   public void testListBucket() throws Exception {
+    if (clientProtocol.equals(RestClient.class)) {
+      return;
+    }
     TestBuckets.runTestListBucket(client);
   }
 }

+ 34 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rest.RestClient;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.OzoneVolume;
@@ -35,14 +36,19 @@ import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Ignore;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.File;
 import java.io.IOException;
 import java.text.ParseException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
@@ -53,9 +59,22 @@ import static org.junit.Assert.assertTrue;
 /**
  * Test Ozone Volumes Lifecycle.
  */
+@RunWith(value = Parameterized.class)
 public class TestVolume {
   private static MiniOzoneCluster cluster = null;
   private static ClientProtocol client = null;
+  private static OzoneConfiguration conf;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> clientProtocol() {
+    Object[][] params = new Object[][] {
+        {RpcClient.class},
+        {RestClient.class}};
+    return Arrays.asList(params);
+  }
+
+  @Parameterized.Parameter
+  public Class clientProtocol;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -68,7 +87,7 @@ public class TestVolume {
    */
   @BeforeClass
   public static void init() throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
+    conf = new OzoneConfiguration();
 
     String path = GenericTestUtils
         .getTempPath(TestVolume.class.getSimpleName());
@@ -81,8 +100,15 @@ public class TestVolume {
 
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
+  }
 
-    client = new RpcClient(conf);
+  @Before
+  public void setup() throws Exception {
+    if (clientProtocol.equals(RestClient.class)) {
+      client = new RestClient(conf);
+    } else {
+      client = new RpcClient(conf);
+    }
   }
 
   /**
@@ -202,6 +228,9 @@ public class TestVolume {
 
   @Test
   public void testListVolume() throws OzoneException, IOException {
+    if (clientProtocol.equals(RestClient.class)) {
+      return;
+    }
     runTestListVolume(client);
   }
 
@@ -286,6 +315,9 @@ public class TestVolume {
 
   @Test
   public void testListVolumes() throws Exception {
+    if (clientProtocol.equals(RestClient.class)) {
+      return;
+    }
     runTestListVolumes(client);
   }
 

+ 27 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java

@@ -26,27 +26,45 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.rest.RestClient;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.*;
 import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
 
 /** The same as {@link TestVolume} except that this test is Ratis enabled. */
 @Ignore("Disabling Ratis tests for pipeline work.")
+@RunWith(value = Parameterized.class)
 public class TestVolumeRatis {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
   private static ClientProtocol client;
   private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> clientProtocol() {
+    Object[][] params = new Object[][] {
+        {RpcClient.class},
+        {RestClient.class}};
+    return Arrays.asList(params);
+  }
+
+  @Parameterized.Parameter
+  public Class clientProtocol;
 
   @BeforeClass
   public static void init() throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
+    conf = new OzoneConfiguration();
 
     // This enables Ratis in the cluster.
     conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, true);
@@ -66,8 +84,15 @@ public class TestVolumeRatis {
     final int port = cluster.getHddsDatanodes().get(0)
         .getDatanodeDetails()
         .getPort(DatanodeDetails.Port.Name.REST).getValue();
+  }
 
-    client = new RpcClient(conf);
+  @Before
+  public void setup() throws Exception {
+    if (clientProtocol.equals(RestClient.class)) {
+      client = new RestClient(conf);
+    } else {
+      client = new RpcClient(conf);
+    }
   }
 
   @AfterClass

+ 12 - 9
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.storage;
 
 import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.client.io.LengthInputStream;
 import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
@@ -55,7 +56,6 @@ import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.response.ListVolumes;
 import org.apache.hadoop.ozone.web.response.VolumeInfo;
 import org.apache.hadoop.ozone.web.response.VolumeOwner;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.response.ListBuckets;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.KeyInfo;
@@ -209,7 +209,8 @@ public final class DistributedStorageHandler implements StorageHandler {
       info.setOwner(new VolumeOwner(infoProto.getOwnerName()));
       info.setQuota(OzoneQuota.getOzoneQuota(infoProto.getQuotaInBytes()));
       info.setVolumeName(infoProto.getVolume());
-      info.setCreatedOn(OzoneUtils.formatTime(infoProto.getCreationTime()));
+      info.setCreatedOn(
+          HddsClientUtils.formatDateTime(infoProto.getCreationTime()));
       result.addVolume(info);
     }
 
@@ -233,7 +234,8 @@ public final class DistributedStorageHandler implements StorageHandler {
             volumeArgs.getAdminName());
     volInfo.setOwner(new VolumeOwner(volumeArgs.getOwnerName()));
     volInfo.setQuota(OzoneQuota.getOzoneQuota(volumeArgs.getQuotaInBytes()));
-    volInfo.setCreatedOn(OzoneUtils.formatTime(volumeArgs.getCreationTime()));
+    volInfo.setCreatedOn(
+        HddsClientUtils.formatDateTime(volumeArgs.getCreationTime()));
     return volInfo;
   }
 
@@ -363,7 +365,8 @@ public final class DistributedStorageHandler implements StorageHandler {
         bk.setBucketName(bucketInfo.getBucketName());
         bk.setStorageType(bucketInfo.getStorageType());
         bk.setAcls(bucketInfo.getAcls());
-        bk.setCreatedOn(OzoneUtils.formatTime(bucketInfo.getCreationTime()));
+        bk.setCreatedOn(
+            HddsClientUtils.formatDateTime(bucketInfo.getCreationTime()));
         result.addBucket(bk);
       }
       return result;
@@ -391,7 +394,7 @@ public final class DistributedStorageHandler implements StorageHandler {
     bucketInfo.setStorageType(ksmBucketInfo.getStorageType());
     bucketInfo.setAcls(ksmBucketInfo.getAcls());
     bucketInfo.setCreatedOn(
-        OzoneUtils.formatTime(ksmBucketInfo.getCreationTime()));
+        HddsClientUtils.formatDateTime(ksmBucketInfo.getCreationTime()));
     return bucketInfo;
   }
 
@@ -481,9 +484,9 @@ public final class DistributedStorageHandler implements StorageHandler {
     keyInfo.setKeyName(ksmKeyInfo.getKeyName());
     keyInfo.setSize(ksmKeyInfo.getDataSize());
     keyInfo.setCreatedOn(
-        OzoneUtils.formatTime(ksmKeyInfo.getCreationTime()));
+        HddsClientUtils.formatDateTime(ksmKeyInfo.getCreationTime()));
     keyInfo.setModifiedOn(
-        OzoneUtils.formatTime(ksmKeyInfo.getModificationTime()));
+        HddsClientUtils.formatDateTime(ksmKeyInfo.getModificationTime()));
     return keyInfo;
   }
 
@@ -524,9 +527,9 @@ public final class DistributedStorageHandler implements StorageHandler {
         tempInfo.setKeyName(info.getKeyName());
         tempInfo.setSize(info.getDataSize());
         tempInfo.setCreatedOn(
-            OzoneUtils.formatTime(info.getCreationTime()));
+            HddsClientUtils.formatDateTime(info.getCreationTime()));
         tempInfo.setModifiedOn(
-            OzoneUtils.formatTime(info.getModificationTime()));
+            HddsClientUtils.formatDateTime(info.getModificationTime()));
 
         result.addKey(tempInfo);
       }