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 bo...@apache.org on 2018/05/29 18:10:52 UTC

[14/50] [abbrv] hadoop git commit: HDDS-45. Removal of old OzoneRestClient. Contributed by Lokesh Jain.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
index 5b67657..a9b8175 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
@@ -23,23 +23,31 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.ozone.OzoneConsts.CHUNK_SIZE;
 import static org.junit.Assert.*;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.RandomStringUtils;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.web.client.OzoneRestClient;
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+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.rpc.RpcClient;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.web.client.OzoneBucket;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
-import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.junit.rules.Timeout;
 
+import java.io.IOException;
+import java.io.InputStream;
+
 /**
  * End-to-end testing of Ozone REST operations.
  */
@@ -52,7 +60,9 @@ public class TestOzoneRestWithMiniCluster {
 
   private static MiniOzoneCluster cluster;
   private static OzoneConfiguration conf;
-  private static OzoneRestClient ozoneClient;
+  private static ClientProtocol client;
+  private static ReplicationFactor replicationFactor = ReplicationFactor.ONE;
+  private static ReplicationType replicationType = ReplicationType.STAND_ALONE;
 
   @Rule
   public ExpectedException exception = ExpectedException.none();
@@ -62,180 +72,125 @@ public class TestOzoneRestWithMiniCluster {
     conf = new OzoneConfiguration();
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
-    int port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
-    ozoneClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
-    ozoneClient.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+    client = new RpcClient(conf);
   }
 
   @AfterClass
-  public static void shutdown() throws InterruptedException {
+  public static void shutdown() throws InterruptedException, IOException {
     if (cluster != null) {
       cluster.shutdown();
     }
-    IOUtils.cleanupWithLogger(null, ozoneClient);
+    client.close();
   }
 
   @Test
   public void testCreateAndGetVolume() throws Exception {
-    String volumeName = nextId("volume");
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    volume = ozoneClient.getVolume(volumeName);
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
+    createAndGetVolume();
   }
 
   @Test
   public void testCreateAndGetBucket() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket = volume.getBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
+    OzoneVolume volume = createAndGetVolume();
+    createAndGetBucket(volume);
   }
 
   @Test
   public void testPutAndGetKey() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     String keyData = nextId("data");
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
+  }
+
+  private void putKey(OzoneBucket bucket, String keyName, String keyData) throws IOException {
+    try (
+        OzoneOutputStream ozoneOutputStream = bucket
+            .createKey(keyName, 0, replicationType, replicationFactor);
+        InputStream inputStream = IOUtils.toInputStream(keyData, UTF_8)) {
+      IOUtils.copy(inputStream, ozoneOutputStream);
+    }
+    try (
+        InputStream inputStream = IOUtils.toInputStream(keyData, UTF_8);
+        OzoneInputStream ozoneInputStream = bucket.readKey(keyName)) {
+      IOUtils.contentEquals(ozoneInputStream, inputStream);
+    }
   }
 
   @Test
   public void testPutAndGetEmptyKey() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     String keyData = "";
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
   }
 
   @Test
   public void testPutAndGetMultiChunkKey() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     int keyDataLen = 3 * CHUNK_SIZE;
     String keyData = buildKeyData(keyDataLen);
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
   }
 
   @Test
   public void testPutAndGetMultiChunkKeyLastChunkPartial() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     int keyDataLen = (int)(2.5 * CHUNK_SIZE);
     String keyData = buildKeyData(keyDataLen);
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
   }
 
   @Test
   public void testReplaceKey() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     int keyDataLen = (int)(2.5 * CHUNK_SIZE);
     String keyData = buildKeyData(keyDataLen);
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
 
     // Replace key with data consisting of fewer chunks.
     keyDataLen = (int)(1.5 * CHUNK_SIZE);
     keyData = buildKeyData(keyDataLen);
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    putKey(bucket, keyName, keyData);
 
     // Replace key with data consisting of more chunks.
     keyDataLen = (int)(3.5 * CHUNK_SIZE);
     keyData = buildKeyData(keyDataLen);
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    putKey(bucket, keyName, keyData);
+  }
+
+  private OzoneVolume createAndGetVolume() throws IOException {
+    String volumeName = nextId("volume");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume volume = client.getVolumeDetails(volumeName);
+    assertEquals(volumeName, volume.getName());
+    assertNotNull(volume);
+    assertEquals("bilbo", volume.getOwner());
+    assertNotNull(volume.getQuota());
+    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
+        volume.getQuota());
+    return volume;
+  }
+
+  private OzoneBucket createAndGetBucket(OzoneVolume vol) throws IOException {
+    String bucketName = nextId("bucket");
+    vol.createBucket(bucketName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    assertNotNull(bucket);
+    assertEquals(bucketName, bucket.getName());
+    return bucket;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
index 64e5f71..684f4d3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
@@ -17,12 +17,17 @@
  */
 package org.apache.hadoop.ozone.web.client;
 
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+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.rpc.RpcClient;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -36,7 +41,10 @@ import org.junit.rules.Timeout;
 import java.io.IOException;
 import java.net.URISyntaxException;
 import java.text.ParseException;
+import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -54,7 +62,7 @@ public class TestBuckets {
   public Timeout testTimeout = new Timeout(300000);
 
   private static MiniOzoneCluster cluster = null;
-  private static OzoneRestClient ozoneRestClient = null;
+  private static ClientProtocol client = null;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -76,11 +84,10 @@ public class TestBuckets {
         OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
 
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    cluster = MiniOzoneCluster.newBuilder(conf).build();
-    final int port = cluster.getHddsDatanodes().get(0).getDatanodeDetails()
-        .getOzoneRestPort();
-    ozoneRestClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(3)
+        .build();
+    client = new RpcClient(conf);
   }
 
   /**
@@ -95,110 +102,151 @@ public class TestBuckets {
 
   @Test
   public void testCreateBucket() throws Exception {
-    runTestCreateBucket(ozoneRestClient);
+    runTestCreateBucket(client);
   }
 
-  static void runTestCreateBucket(OzoneRestClient client)
+  static void runTestCreateBucket(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
 
     // create 10 buckets under same volume
     for (int x = 0; x < 10; x++) {
       long currentTime = Time.now();
       String bucketName = OzoneUtils.getRequestID().toLowerCase();
-      OzoneBucket bucket =
-          vol.createBucket(bucketName, acls, StorageType.DEFAULT);
-      assertEquals(bucket.getBucketName(), bucketName);
+
+      List<OzoneAcl> aclList =
+          Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+              .collect(Collectors.toList());
+      BucketArgs bucketArgs = BucketArgs.newBuilder()
+          .setAcls(aclList)
+          .build();
+      vol.createBucket(bucketName, bucketArgs);
+      OzoneBucket bucket = vol.getBucket(bucketName);
+      assertEquals(bucket.getName(), bucketName);
 
       // verify the bucket creation time
-      assertTrue((OzoneUtils.formatDate(bucket.getCreatedOn())
-          / 1000) >= (currentTime / 1000));
+      assertTrue((bucket.getCreationTime() / 1000) >= (currentTime / 1000));
     }
     client.close();
 
-    assertEquals(vol.getVolumeName(), volumeName);
-    assertEquals(vol.getCreatedby(), "hdfs");
-    assertEquals(vol.getOwnerName(), "bilbo");
-    assertEquals(vol.getQuota().getUnit(), OzoneQuota.Units.TB);
-    assertEquals(vol.getQuota().getSize(), 100);
+    assertEquals(vol.getName(), volumeName);
+    assertEquals(vol.getAdmin(), "hdfs");
+    assertEquals(vol.getOwner(), "bilbo");
+    assertEquals(vol.getQuota(), OzoneQuota.parseQuota("100TB").sizeInBytes());
 
     // Test create a bucket with invalid bucket name,
     // not use Rule here because the test method is static.
     try {
       String invalidBucketName = "#" + OzoneUtils.getRequestID().toLowerCase();
-      vol.createBucket(invalidBucketName, acls, StorageType.DEFAULT);
+      vol.createBucket(invalidBucketName);
       fail("Except the bucket creation to be failed because the"
           + " bucket name starts with an invalid char #");
     } catch (Exception e) {
-      assertTrue(e instanceof OzoneClientException);
-      assertTrue(e.getMessage().contains("Bucket or Volume name"
-          + " has an unsupported character : #"));
+      assertTrue(e.getMessage()
+          .contains("Bucket or Volume name has an unsupported character : #"));
     }
   }
 
   @Test
   public void testAddBucketAcls() throws Exception {
-    runTestAddBucketAcls(ozoneRestClient);
+    runTestAddBucketAcls(client);
   }
 
-  static void runTestAddBucketAcls(OzoneRestClient client)
+  static void runTestAddBucketAcls(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
     vol.createBucket(bucketName);
-    vol.addAcls(bucketName, acls);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    List<OzoneAcl> aclList =
+        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+            .collect(Collectors.toList());
+    int numAcls = bucket.getAcls().size();
+    bucket.addAcls(aclList);
     OzoneBucket updatedBucket = vol.getBucket(bucketName);
-    assertEquals(updatedBucket.getAcls().size(), 2);
+    assertEquals(updatedBucket.getAcls().size(), 2 + numAcls);
     // verify if the creation time is missing after update operation
     assertTrue(
-        (OzoneUtils.formatDate(updatedBucket.getCreatedOn()) / 1000) >= 0);
+        (updatedBucket.getCreationTime()) / 1000 >= 0);
     client.close();
   }
 
   @Test
   public void testRemoveBucketAcls() throws Exception {
-    runTestRemoveBucketAcls(ozoneRestClient);
+    runTestRemoveBucketAcls(client);
   }
 
-  static void runTestRemoveBucketAcls(OzoneRestClient client)
+  static void runTestRemoveBucketAcls(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
-    OzoneBucket bucket = vol.createBucket(bucketName, acls);
-    assertEquals(bucket.getAcls().size(), 2);
-    vol.removeAcls(bucketName, acls);
+    List<OzoneAcl> aclList =
+        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+            .collect(Collectors.toList());
+    vol.createBucket(bucketName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    int numAcls = bucket.getAcls().size();
+    bucket.addAcls(aclList);
+    assertEquals(bucket.getAcls().size(), 2 + numAcls);
+    bucket.removeAcls(aclList);
     OzoneBucket updatedBucket = vol.getBucket(bucketName);
 
     // We removed all acls
-    assertEquals(updatedBucket.getAcls().size(), 0);
+    assertEquals(updatedBucket.getAcls().size(), numAcls);
     // verify if the creation time is missing after update operation
     assertTrue(
-        (OzoneUtils.formatDate(updatedBucket.getCreatedOn()) / 1000) >= 0);
+        (updatedBucket.getCreationTime() / 1000) >= 0);
     client.close();
   }
 
   @Test
   public void testDeleteBucket() throws OzoneException, IOException {
-    runTestDeleteBucket(ozoneRestClient);
+    runTestDeleteBucket(client);
   }
 
-  static void runTestDeleteBucket(OzoneRestClient client)
+  static void runTestDeleteBucket(ClientProtocol client)
       throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
-    vol.createBucket(bucketName, acls);
+    List<OzoneAcl> aclList =
+        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+            .collect(Collectors.toList());
+    BucketArgs bucketArgs = BucketArgs.newBuilder()
+        .setAcls(aclList)
+        .build();
+    vol.createBucket(bucketName, bucketArgs);
     vol.deleteBucket(bucketName);
     try {
       OzoneBucket updatedBucket = vol.getBucket(bucketName);
@@ -212,38 +260,57 @@ public class TestBuckets {
 
   @Test
   public void testListBucket() throws Exception {
-    runTestListBucket(ozoneRestClient);
+    runTestListBucket(client);
   }
 
-  static void runTestListBucket(OzoneRestClient client)
+  static void runTestListBucket(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
+    List<OzoneAcl> aclList =
+        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+            .collect(Collectors.toList());
 
     long currentTime = Time.now();
     for (int x = 0; x < 10; x++) {
       String bucketName = "listbucket-test-" + x;
-      vol.createBucket(bucketName, acls);
+      BucketArgs bucketArgs = BucketArgs.newBuilder()
+          .setAcls(aclList)
+          .build();
+      vol.createBucket(bucketName, bucketArgs);
     }
-    List<OzoneBucket> bucketList = vol.listBuckets("100", null, null);
-    assertEquals(bucketList.size(), 10);
+    Iterator<OzoneBucket> bucketIterator = vol.listBuckets(null);
+    int count = 0;
 
-    for (OzoneBucket bucket : bucketList) {
-      assertTrue((OzoneUtils.formatDate(bucket.getCreatedOn())
+    while (bucketIterator.hasNext()) {
+      assertTrue((bucketIterator.next().getCreationTime()
           / 1000) >= (currentTime / 1000));
+      count++;
     }
+    assertEquals(count, 10);
 
-    bucketList = vol.listBuckets("3", null, null);
-    assertEquals(bucketList.size(), 3);
-
-    bucketList = vol.listBuckets("100", "listbucket-test-4", null);
-    assertEquals(bucketList.size(), 5);
+    bucketIterator = vol.listBuckets(null, "listbucket-test-4");
+    assertEquals(getSize(bucketIterator), 5);
 
-    bucketList = vol.listBuckets("100", null, "listbucket-test-3");
-    assertEquals(bucketList.size(), 1);
+    bucketIterator = vol.listBuckets(null, "listbucket-test-3");
+    assertEquals(getSize(bucketIterator), 6);
 
     client.close();
   }
+
+  private static int getSize(Iterator<OzoneBucket> bucketIterator) {
+    int count = 0;
+    while (bucketIterator.hasNext()) {
+      count++;
+      bucketIterator.next();
+    }
+    return count;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
index b913a86..9f80184 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.web.client;
 
 import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -35,12 +36,12 @@ public class TestBucketsRatis {
   public Timeout testTimeout = new Timeout(300000);
 
   private static RatisTestHelper.RatisTestSuite suite;
-  private static OzoneRestClient ozoneRestClient;
+  private static ClientProtocol client;
 
   @BeforeClass
   public static void init() throws Exception {
     suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
-    ozoneRestClient = suite.newOzoneRestClient();
+    client = suite.newOzoneClient();
   }
 
   @AfterClass
@@ -52,25 +53,25 @@ public class TestBucketsRatis {
 
   @Test
   public void testCreateBucket() throws Exception {
-    TestBuckets.runTestCreateBucket(ozoneRestClient);
+    TestBuckets.runTestCreateBucket(client);
   }
 
   @Test
   public void testAddBucketAcls() throws Exception {
-    TestBuckets.runTestAddBucketAcls(ozoneRestClient);
+    TestBuckets.runTestAddBucketAcls(client);
   }
 
   @Test
   public void testRemoveBucketAcls() throws Exception {
-    TestBuckets.runTestRemoveBucketAcls(ozoneRestClient);
+    TestBuckets.runTestRemoveBucketAcls(client);
   }
 
   @Test
   public void testDeleteBucket() throws OzoneException, IOException {
-    TestBuckets.runTestDeleteBucket(ozoneRestClient);
+    TestBuckets.runTestDeleteBucket(client);
   }
   @Test
   public void testListBucket() throws Exception {
-    TestBuckets.runTestListBucket(ozoneRestClient);
+    TestBuckets.runTestListBucket(client);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index 57d4287..f8df7dc 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -20,16 +20,29 @@ package org.apache.hadoop.ozone.web.client;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.collections.IteratorUtils;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.lang.math.RandomUtils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
@@ -59,15 +72,18 @@ import org.junit.rules.Timeout;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.InputStream;
 import java.io.IOException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -84,10 +100,13 @@ public class TestKeys {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
 
+  private static OzoneConfiguration conf;
   private static MiniOzoneCluster ozoneCluster = null;
   private static String path;
-  private static OzoneRestClient ozoneRestClient = null;
+  private static ClientProtocol client = null;
   private static long currentTime;
+  private static ReplicationFactor replicationFactor = ReplicationFactor.ONE;
+  private static ReplicationType replicationType = ReplicationType.STAND_ALONE;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -96,7 +115,7 @@ public class TestKeys {
    */
   @BeforeClass
   public static void init() throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
+    conf = new OzoneConfiguration();
 
     // Set short block deleting service interval to speed up deletions.
     conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
@@ -105,12 +124,9 @@ public class TestKeys {
     path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
-    ozoneCluster = MiniOzoneCluster.newBuilder(conf).build();
+    ozoneCluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
     ozoneCluster.waitForClusterToBeReady();
-    final int port = ozoneCluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
-    ozoneRestClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
+    client = new RpcClient(conf);
     currentTime = Time.now();
   }
 
@@ -166,7 +182,7 @@ public class TestKeys {
   }
 
   static class PutHelper {
-    private final OzoneRestClient client;
+    private final ClientProtocol client;
     private final String dir;
     private final String keyName;
 
@@ -174,11 +190,11 @@ public class TestKeys {
     private OzoneBucket bucket;
     private File file;
 
-    PutHelper(OzoneRestClient client, String dir) {
+    PutHelper(ClientProtocol client, String dir) {
       this(client, dir, OzoneUtils.getRequestID().toLowerCase());
     }
 
-    PutHelper(OzoneRestClient client, String dir, String key) {
+    PutHelper(ClientProtocol client, String dir, String key) {
       this.client = client;
       this.dir = dir;
       this.keyName = key;
@@ -202,111 +218,139 @@ public class TestKeys {
      * @return Returns the name of the new key that was created.
      * @throws OzoneException
      */
-    private KsmKeyArgs putKey() throws Exception {
+    private String putKey() throws Exception {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
-      client.setUserAuth("hdfs");
 
-      vol = client.createVolume(volumeName, "bilbo", "100TB");
+      VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+          .setOwner("bilbo")
+          .setQuota("100TB")
+          .setAdmin("hdfs")
+          .build();
+      client.createVolume(volumeName, volumeArgs);
+      vol = client.getVolumeDetails(volumeName);
       String[] acls = {"user:frodo:rw", "user:samwise:rw"};
 
       String bucketName = OzoneUtils.getRequestID().toLowerCase();
-      bucket = vol.createBucket(bucketName, acls, StorageType.DEFAULT);
+      List<OzoneAcl> aclList =
+          Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+              .collect(Collectors.toList());
+      BucketArgs bucketArgs = BucketArgs.newBuilder()
+          .setAcls(aclList)
+          .build();
+      vol.createBucket(bucketName, bucketArgs);
+      bucket = vol.getBucket(bucketName);
 
       String fileName = OzoneUtils.getRequestID().toLowerCase();
 
       file = createRandomDataFile(dir, fileName, 1024);
 
-      bucket.putKey(keyName, file);
-      return new KsmKeyArgs.Builder()
-          .setKeyName(keyName)
-          .setVolumeName(volumeName)
-          .setBucketName(bucketName)
-          .setDataSize(1024)
-          .build();
+      try (
+          OzoneOutputStream ozoneOutputStream = bucket
+              .createKey(keyName, 0, replicationType, replicationFactor);
+          InputStream fileInputStream = new FileInputStream(file)) {
+        IOUtils.copy(fileInputStream, ozoneOutputStream);
+      }
+      return keyName;
     }
   }
 
   @Test
   public void testPutKey() throws Exception {
     // Test non-delimited keys
-    runTestPutKey(new PutHelper(ozoneRestClient, path));
+    runTestPutKey(new PutHelper(client, path));
     // Test key delimited by a random delimiter
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestPutKey(new PutHelper(ozoneRestClient, path,
+    runTestPutKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestPutKey(PutHelper helper) throws Exception {
-    final OzoneRestClient client = helper.client;
+    final ClientProtocol client = helper.client;
     helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
-    List<OzoneKey> keyList = helper.getBucket().listKeys("100", null, null);
+    List<OzoneKey> keyList = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
+            null, 10);
     Assert.assertEquals(1, keyList.size());
 
     // test list key using a more efficient call
     String newkeyName = OzoneUtils.getRequestID().toLowerCase();
-    client.putKey(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), newkeyName, helper.getFile());
-    keyList = helper.getBucket().listKeys("100", null, null);
+    OzoneOutputStream ozoneOutputStream = client
+        .createKey(helper.getVol().getName(), helper.getBucket().getName(),
+            newkeyName, 0, replicationType, replicationFactor);
+    ozoneOutputStream.close();
+    keyList = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
+            null, 10);
     Assert.assertEquals(2, keyList.size());
 
     // test new put key with invalid volume/bucket name
     try {
-      client.putKey("invalid-volume",
-          helper.getBucket().getBucketName(), newkeyName, helper.getFile());
+      ozoneOutputStream = client
+          .createKey("invalid-volume", helper.getBucket().getName(), newkeyName,
+              0, replicationType, replicationFactor);
+      ozoneOutputStream.close();
       fail("Put key should have thrown"
           + " when using invalid volume name.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           Status.VOLUME_NOT_FOUND.toString(), e);
     }
 
     try {
-      client.putKey(helper.getVol().getVolumeName(), "invalid-bucket",
-          newkeyName, helper.getFile());
+      ozoneOutputStream = client
+          .createKey(helper.getVol().getName(), "invalid-bucket", newkeyName, 0,
+              replicationType, replicationFactor);
+      ozoneOutputStream.close();
       fail("Put key should have thrown "
           + "when using invalid bucket name.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           Status.BUCKET_NOT_FOUND.toString(), e);
     }
   }
 
-  private static void restartDatanode(
-      MiniOzoneCluster cluster, int datanodeIdx, OzoneRestClient client)
+  private static void restartDatanode(MiniOzoneCluster cluster, int datanodeIdx)
       throws OzoneException, URISyntaxException {
     cluster.restartHddsDatanode(datanodeIdx);
-    // refresh the datanode endpoint uri after datanode restart
-    final int port = ozoneCluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
-    client.setEndPoint(String.format("http://localhost:%d", port));
   }
 
   @Test
   public void testPutAndGetKeyWithDnRestart() throws Exception {
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path), ozoneCluster);
+        new PutHelper(client, path), ozoneCluster);
     String delimiter = RandomStringUtils.randomAscii(1);
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path,
+        new PutHelper(client, path,
             getMultiPartKey(delimiter)), ozoneCluster);
   }
 
   static void runTestPutAndGetKeyWithDnRestart(
       PutHelper helper, MiniOzoneCluster cluster) throws Exception {
-    String keyName = helper.putKey().getKeyName();
+    String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
     // restart the datanode
-    restartDatanode(cluster, 0, helper.client);
+    restartDatanode(cluster, 0);
+    // TODO: Try removing sleep and adding a join for the MiniOzoneCluster start
+    // The ozoneContainer is not started and its metrics are not initialized
+    // which leads to NullPointerException in Dispatcher.
+    Thread.sleep(1000);
+    ozoneCluster.waitForClusterToBeReady();
     // verify getKey after the datanode restart
     String newFileName = helper.dir + "/"
         + OzoneUtils.getRequestID().toLowerCase();
     Path newPath = Paths.get(newFileName);
-
-    helper.getBucket().getKey(keyName, newPath);
+    try (
+        FileOutputStream newOutputStream = new FileOutputStream(
+            newPath.toString());
+        OzoneInputStream ozoneInputStream = helper.client
+            .getKey(helper.getVol().getName(), helper.getBucket().getName(),
+                keyName)) {
+      IOUtils.copy(ozoneInputStream, newOutputStream);
+    }
 
     try (
         FileInputStream original = new FileInputStream(helper.getFile());
@@ -321,16 +365,16 @@ public class TestKeys {
 
   @Test
   public void testPutAndGetKey() throws Exception {
-    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndGetKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndGetKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestPutAndGetKey(PutHelper helper) throws Exception {
-    final OzoneRestClient client = helper.client;
+    final ClientProtocol client = helper.client;
 
-    String keyName = helper.putKey().getKeyName();
+    String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
@@ -342,10 +386,22 @@ public class TestKeys {
     Path newPath1 = Paths.get(newFileName1);
     Path newPath2 = Paths.get(newFileName2);
 
-    helper.getBucket().getKey(keyName, newPath1);
+    try (
+        FileOutputStream newOutputStream = new FileOutputStream(
+            newPath1.toString());
+        OzoneInputStream ozoneInputStream = helper.getBucket()
+            .readKey(keyName)) {
+      IOUtils.copy(ozoneInputStream, newOutputStream);
+    }
+
     // test get key using a more efficient call
-    client.getKey(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), keyName, newPath2);
+    try (
+        FileOutputStream newOutputStream = new FileOutputStream(
+            newPath2.toString());
+        OzoneInputStream ozoneInputStream = helper.getBucket()
+            .readKey(keyName)) {
+      IOUtils.copy(ozoneInputStream, newOutputStream);
+    }
 
     try (FileInputStream original = new FileInputStream(helper.getFile());
         FileInputStream downloaded1 = new FileInputStream(newPath1.toFile());
@@ -363,19 +419,17 @@ public class TestKeys {
 
       // test new get key with invalid volume/bucket name
       try {
-        client.getKey("invalid-volume", helper.getBucket().getBucketName(),
-            keyName, newPath1);
+        client.getKey("invalid-volume", helper.getBucket().getName(), keyName);
         fail("Get key should have thrown " + "when using invalid volume name.");
-      } catch (OzoneException e) {
+      } catch (IOException e) {
         GenericTestUtils
             .assertExceptionContains(Status.KEY_NOT_FOUND.toString(), e);
       }
 
       try {
-        client.getKey(helper.getVol().getVolumeName(), "invalid-bucket",
-            keyName, newPath1);
+        client.getKey(helper.getVol().getName(), "invalid-bucket", keyName);
         fail("Get key should have thrown " + "when using invalid bucket name.");
-      } catch (OzoneException e) {
+      } catch (IOException e) {
         GenericTestUtils.assertExceptionContains(
             Status.KEY_NOT_FOUND.toString(), e);
       }
@@ -384,14 +438,14 @@ public class TestKeys {
 
   @Test
   public void testPutAndDeleteKey() throws Exception {
-    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndDeleteKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndDeleteKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestPutAndDeleteKey(PutHelper helper) throws Exception {
-    String keyName = helper.putKey().getKeyName();
+    String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
     helper.getBucket().deleteKey(keyName);
@@ -399,7 +453,7 @@ public class TestKeys {
     try {
       helper.getBucket().getKey(keyName);
       fail("Get Key on a deleted key should have thrown");
-    } catch (OzoneException ex) {
+    } catch (IOException ex) {
       GenericTestUtils.assertExceptionContains(
           Status.KEY_NOT_FOUND.toString(), ex);
     }
@@ -407,14 +461,14 @@ public class TestKeys {
 
   @Test
   public void testPutAndListKey() throws Exception {
-    runTestPutAndListKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndListKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestPutAndListKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndListKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestPutAndListKey(PutHelper helper) throws Exception {
-    final OzoneRestClient client = helper.client;
+    ClientProtocol client = helper.client;
     helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
@@ -422,13 +476,20 @@ public class TestKeys {
     // add keys [list-key0, list-key1, ..., list-key9]
     for (int x = 0; x < 10; x++) {
       String newkeyName = "list-key" + x;
-      helper.getBucket().putKey(newkeyName, helper.getFile());
+      try (
+          OzoneOutputStream ozoneOutputStream = helper.getBucket()
+              .createKey(newkeyName, 0, replicationType, replicationFactor);
+          InputStream fileInputStream = new FileInputStream(helper.getFile())) {
+        IOUtils.copy(fileInputStream, ozoneOutputStream);
+      }
     }
 
-    List<OzoneKey> keyList1 = helper.getBucket().listKeys("100", null, null);
+    List<OzoneKey> keyList1 =
+        IteratorUtils.toList(helper.getBucket().listKeys(null, null));
     // test list key using a more efficient call
-    List<OzoneKey> keyList2 = client.listKeys(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), "100", null, null);
+    List<OzoneKey> keyList2 = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
+            null, 100);
 
     Assert.assertEquals(11, keyList1.size());
     Assert.assertEquals(11, keyList2.size());
@@ -436,57 +497,56 @@ public class TestKeys {
     // second unit since the date string reparsed to millisecond will
     // lose precision.
     for (OzoneKey key : keyList1) {
-      assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getCreatedOn())
-          / 1000) >= (currentTime / 1000));
-      assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getModifiedOn())
-          / 1000) >= (currentTime / 1000));
+      assertTrue((key.getCreationTime() / 1000) >= (currentTime / 1000));
+      assertTrue((key.getModificationTime() / 1000) >= (currentTime / 1000));
     }
 
     for (OzoneKey key : keyList2) {
-      assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getCreatedOn())
-          / 1000) >= (currentTime / 1000));
-      assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getModifiedOn())
-          / 1000) >= (currentTime / 1000));
+      assertTrue((key.getCreationTime() / 1000) >= (currentTime / 1000));
+      assertTrue((key.getModificationTime() / 1000) >= (currentTime / 1000));
     }
 
     // test maxLength parameter of list keys
-    keyList1 = helper.getBucket().listKeys("1", null, null);
-    keyList2 = client.listKeys(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), "1", null, null);
-    Assert.assertEquals(1, keyList1.size());
+    keyList2 = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
+            null, 1);
     Assert.assertEquals(1, keyList2.size());
 
     // test startKey parameter of list keys
-    keyList1 = helper.getBucket().listKeys("100", "list-key4", "list-key");
-    keyList2 = client.listKeys(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), "100", "list-key4", "list-key");
+    keyList1 = IteratorUtils
+        .toList(helper.getBucket().listKeys("list-key", "list-key4"));
+    keyList2 = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
+            "list-key", "list-key4", 100);
     Assert.assertEquals(5, keyList1.size());
     Assert.assertEquals(5, keyList2.size());
 
     // test prefix parameter of list keys
-    keyList1 = helper.getBucket().listKeys("100", null, "list-key2");
-    keyList2 = client.listKeys(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), "100", null, "list-key2");
-    Assert.assertTrue(keyList1.size() == 1
-        && keyList1.get(0).getObjectInfo().getKeyName().equals("list-key2"));
-    Assert.assertTrue(keyList2.size() == 1
-        && keyList2.get(0).getObjectInfo().getKeyName().equals("list-key2"));
+    keyList1 =
+        IteratorUtils.toList(helper.getBucket().listKeys("list-key2", null));
+    keyList2 = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
+            "list-key2", null, 100);
+    Assert.assertTrue(
+        keyList1.size() == 1 && keyList1.get(0).getName().equals("list-key2"));
+    Assert.assertTrue(
+        keyList2.size() == 1 && keyList2.get(0).getName().equals("list-key2"));
 
     // test new list keys with invalid volume/bucket name
     try {
-      client.listKeys("invalid-volume", helper.getBucket().getBucketName(),
-          "100", null, null);
+      client.listKeys("invalid-volume", helper.getBucket().getName(),
+          null, null, 100);
       fail("List keys should have thrown when using invalid volume name.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           Status.BUCKET_NOT_FOUND.toString(), e);
     }
 
     try {
-      client.listKeys(helper.getVol().getVolumeName(), "invalid-bucket", "100",
-          null, null);
+      client.listKeys(helper.getVol().getName(), "invalid-bucket", null,
+          null, 100);
       fail("List keys should have thrown when using invalid bucket name.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           Status.BUCKET_NOT_FOUND.toString(), e);
     }
@@ -494,29 +554,27 @@ public class TestKeys {
 
   @Test
   public void testGetKeyInfo() throws Exception {
-    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path));
+    runTestGetKeyInfo(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path,
+    runTestGetKeyInfo(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestGetKeyInfo(PutHelper helper) throws Exception {
-    String keyName = helper.putKey().getKeyName();
+    String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
-    OzoneKey keyInfo = helper.getBucket().getKeyInfo(keyName);
-    assertNotNull(keyInfo.getObjectInfo());
-    assertEquals(keyName, keyInfo.getObjectInfo().getKeyName());
+    OzoneKey keyInfo = helper.getBucket().getKey(keyName);
+    assertNotNull(keyInfo);
+    assertEquals(keyName, keyInfo.getName());
 
     // Compare the time in second unit since the date string reparsed to
     // millisecond will lose precision.
+    Assert
+        .assertTrue((keyInfo.getCreationTime() / 1000) >= (currentTime / 1000));
     Assert.assertTrue(
-        (OzoneUtils.formatDate(keyInfo.getObjectInfo().getCreatedOn())
-            / 1000) >= (currentTime / 1000));
-    Assert.assertTrue(
-        (OzoneUtils.formatDate(keyInfo.getObjectInfo().getModifiedOn())
-            / 1000) >= (currentTime / 1000));
+        (keyInfo.getModificationTime() / 1000) >= (currentTime / 1000));
   }
 
   // Volume, bucket, keys info that helps for test create/delete keys.
@@ -593,12 +651,12 @@ public class TestKeys {
     int numOfExistedKeys = countKsmKeys(ksm);
 
     // Keep tracking bucket keys info while creating them
-    PutHelper helper = new PutHelper(ozoneRestClient, path);
+    PutHelper helper = new PutHelper(client, path);
     BucketKeys bucketKeys = new BucketKeys();
     for (int i = 0; i < 20; i++) {
-      KsmKeyArgs keyArgs = helper.putKey();
-      bucketKeys.addKey(keyArgs.getVolumeName(), keyArgs.getBucketName(),
-          keyArgs.getKeyName());
+      String keyName = helper.putKey();
+      bucketKeys.addKey(helper.getVol().getName(), helper.getBucket().getName(),
+          keyName);
     }
 
     // There should be 20 keys in the buckets we just created.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
index 802cc3d..645b866 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
@@ -21,6 +21,7 @@ import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -52,7 +53,7 @@ public class TestKeysRatis {
   private static RatisTestHelper.RatisTestSuite suite;
   private static MiniOzoneCluster ozoneCluster = null;
   static private String path;
-  private static OzoneRestClient ozoneRestClient = null;
+  private static ClientProtocol client = null;
 
   @BeforeClass
   public static void init() throws Exception {
@@ -60,7 +61,7 @@ public class TestKeysRatis {
     path = suite.getConf().get(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT);
     ozoneCluster = suite.getCluster();
     ozoneCluster.waitForClusterToBeReady();
-    ozoneRestClient = suite.newOzoneRestClient();
+    client = suite.newOzoneClient();
   }
 
   /**
@@ -76,9 +77,9 @@ public class TestKeysRatis {
 
   @Test
   public void testPutKey() throws Exception {
-    runTestPutKey(new PutHelper(ozoneRestClient, path));
+    runTestPutKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestPutKey(new PutHelper(ozoneRestClient, path,
+    runTestPutKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
@@ -86,42 +87,42 @@ public class TestKeysRatis {
   @Test
   public void testPutAndGetKeyWithDnRestart() throws Exception {
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path), ozoneCluster);
+        new PutHelper(client, path), ozoneCluster);
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path, getMultiPartKey(delimiter)),
+        new PutHelper(client, path, getMultiPartKey(delimiter)),
         ozoneCluster);
   }
 
   @Test
   public void testPutAndGetKey() throws Exception {
-    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndGetKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndGetKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   @Test
   public void testPutAndDeleteKey() throws Exception  {
-    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndDeleteKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndDeleteKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   @Test
   public void testPutAndListKey() throws Exception {
-    runTestPutAndListKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndListKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestPutAndListKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndListKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   @Test
   public void testGetKeyInfo() throws Exception {
-    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path));
+    runTestGetKeyInfo(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path,
+    runTestGetKeyInfo(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
index 2d3cea9..f8c7eec 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -20,50 +20,42 @@ package org.apache.hadoop.ozone.web.client;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.RandomStringUtils;
-import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.Status;
-import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.web.request.OzoneQuota;
+import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
-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.mockito.Mockito;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.Ignore;
 
 import java.io.File;
 import java.io.IOException;
 import java.text.ParseException;
-import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Test;
-import org.junit.Ignore;
-import static org.junit.Assert.fail;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 
 /**
  * Test Ozone Volumes Lifecycle.
  */
 public class TestVolume {
   private static MiniOzoneCluster cluster = null;
-  private static OzoneRestClient ozoneRestClient = null;
+  private static ClientProtocol client = null;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -92,8 +84,7 @@ public class TestVolume {
     final int port = cluster.getHddsDatanodes().get(0)
         .getDatanodeDetails().getOzoneRestPort();
 
-    ozoneRestClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
+    client = new RpcClient(conf);
   }
 
   /**
@@ -108,128 +99,122 @@ public class TestVolume {
 
   @Test
   public void testCreateVolume() throws Exception {
-    runTestCreateVolume(ozoneRestClient);
+    runTestCreateVolume(client);
   }
 
-  static void runTestCreateVolume(OzoneRestClient client)
+  static void runTestCreateVolume(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
 
     long currentTime = Time.now();
-    OzoneRestClient mockClient = Mockito.spy(client);
-    List<CloseableHttpClient> mockedClients = mockHttpClients(mockClient);
-    OzoneVolume vol = mockClient.createVolume(volumeName, "bilbo", "100TB");
-    // Verify http clients are properly closed.
-    verifyHttpConnectionClosed(mockedClients);
-
-    assertEquals(vol.getVolumeName(), volumeName);
-    assertEquals(vol.getCreatedby(), "hdfs");
-    assertEquals(vol.getOwnerName(), "bilbo");
-    assertEquals(vol.getQuota().getUnit(), OzoneQuota.Units.TB);
-    assertEquals(vol.getQuota().getSize(), 100);
+
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
+
+    assertEquals(vol.getName(), volumeName);
+    assertEquals(vol.getAdmin(), "hdfs");
+    assertEquals(vol.getOwner(), "bilbo");
+    assertEquals(vol.getQuota(), OzoneQuota.parseQuota("100TB").sizeInBytes());
 
     // verify the key creation time
-    assertTrue((OzoneUtils.formatDate(vol.getCreatedOn())
+    assertTrue((vol.getCreationTime()
         / 1000) >= (currentTime / 1000));
 
     // Test create a volume with invalid volume name,
     // not use Rule here because the test method is static.
     try {
       String invalidVolumeName = "#" + OzoneUtils.getRequestID().toLowerCase();
-      client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-      mockClient.createVolume(invalidVolumeName, "bilbo", "100TB");
+      client.createVolume(invalidVolumeName);
+      /*
+      //TODO: RestClient and RpcClient should use HddsClientUtils to verify name
       fail("Except the volume creation be failed because the"
-          + " volume name starts with an invalid char #");
+          + " volume name starts with an invalid char #");*/
     } catch (Exception e) {
-      assertTrue(e instanceof OzoneClientException);
       assertTrue(e.getMessage().contains("Bucket or Volume name"
           + " has an unsupported character : #"));
     }
   }
 
   @Test
-  public void testCreateDuplicateVolume() throws OzoneException {
-    runTestCreateDuplicateVolume(ozoneRestClient);
+  public void testCreateDuplicateVolume() throws OzoneException, IOException {
+    runTestCreateDuplicateVolume(client);
   }
 
-  static void runTestCreateDuplicateVolume(OzoneRestClient client)
-      throws OzoneException {
+  static void runTestCreateDuplicateVolume(ClientProtocol client)
+      throws OzoneException, IOException {
     try {
-      client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-      client.createVolume("testvol", "bilbo", "100TB");
-      client.createVolume("testvol", "bilbo", "100TB");
+      client.createVolume("testvol");
+      client.createVolume("testvol");
       assertFalse(true);
-    } catch (OzoneException ex) {
-      // Ozone will throw saying volume already exists
-      GenericTestUtils.assertExceptionContains(
-          Status.VOLUME_ALREADY_EXISTS.toString(), ex);
+    } catch (IOException ioe) {
+      Assert.assertTrue(ioe.getMessage()
+          .contains("Volume creation failed, error:VOLUME_ALREADY_EXISTS"));
     }
   }
 
   @Test
-  public void testDeleteVolume() throws OzoneException {
-    runTestDeleteVolume(ozoneRestClient);
+  public void testDeleteVolume() throws OzoneException, IOException {
+    runTestDeleteVolume(client);
   }
 
-  static void runTestDeleteVolume(OzoneRestClient client)
-      throws OzoneException {
+  static void runTestDeleteVolume(ClientProtocol client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
-    client.deleteVolume(vol.getVolumeName());
+    client.createVolume(volumeName);
+    client.deleteVolume(volumeName);
   }
 
   @Test
   public void testChangeOwnerOnVolume() throws Exception {
-    runTestChangeOwnerOnVolume(ozoneRestClient);
+    runTestChangeOwnerOnVolume(client);
   }
 
-  static void runTestChangeOwnerOnVolume(OzoneRestClient client)
-      throws OzoneException, ParseException {
+  static void runTestChangeOwnerOnVolume(ClientProtocol client)
+      throws OzoneException, ParseException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    client.createVolume(volumeName);
+    client.getVolumeDetails(volumeName);
     client.setVolumeOwner(volumeName, "frodo");
-    OzoneVolume newVol = client.getVolume(volumeName);
-    assertEquals(newVol.getOwnerName(), "frodo");
+    OzoneVolume newVol = client.getVolumeDetails(volumeName);
+    assertEquals(newVol.getOwner(), "frodo");
     // verify if the creation time is missing after setting owner operation
-    assertTrue(OzoneUtils.formatDate(newVol.getCreatedOn()) > 0);
+    assertTrue(newVol.getCreationTime() > 0);
   }
 
   @Test
   public void testChangeQuotaOnVolume() throws Exception {
-    runTestChangeQuotaOnVolume(ozoneRestClient);
+    runTestChangeQuotaOnVolume(client);
   }
 
-  static void runTestChangeQuotaOnVolume(OzoneRestClient client)
+  static void runTestChangeQuotaOnVolume(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
-    client.setVolumeQuota(volumeName, "1000MB");
-    OzoneVolume newVol = client.getVolume(volumeName);
-    assertEquals(newVol.getQuota().getSize(), 1000);
-    assertEquals(newVol.getQuota().getUnit(), OzoneQuota.Units.MB);
+    client.createVolume(volumeName);
+    client.setVolumeQuota(volumeName, OzoneQuota.parseQuota("1000MB"));
+    OzoneVolume newVol = client.getVolumeDetails(volumeName);
+    assertEquals(newVol.getQuota(), OzoneQuota.parseQuota("1000MB").sizeInBytes());
     // verify if the creation time is missing after setting quota operation
-    assertTrue(OzoneUtils.formatDate(newVol.getCreatedOn()) > 0);
+    assertTrue(newVol.getCreationTime() > 0);
   }
 
   @Test
   public void testListVolume() throws OzoneException, IOException {
-    runTestListVolume(ozoneRestClient);
+    runTestListVolume(client);
   }
 
-  static void runTestListVolume(OzoneRestClient client)
+  static void runTestListVolume(ClientProtocol client)
       throws OzoneException, IOException {
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     for (int x = 0; x < 10; x++) {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
-      OzoneVolume vol = client.createVolume(volumeName, "frodo", "100TB");
-      assertNotNull(vol);
+      client.createVolume(volumeName);
     }
 
-    List<OzoneVolume> ovols = client.listVolumes("frodo");
+    List<OzoneVolume> ovols = client.listVolumes(null, null, 100);
     assertTrue(ovols.size() >= 10);
   }
 
@@ -237,27 +222,24 @@ public class TestVolume {
   @Ignore("Takes 3m to run, disable for now.")
   @Test
   public void testListVolumePagination() throws OzoneException, IOException {
-    runTestListVolumePagination(ozoneRestClient);
+    runTestListVolumePagination(client);
   }
 
-  static void runTestListVolumePagination(OzoneRestClient client)
+  static void runTestListVolumePagination(ClientProtocol client)
       throws OzoneException, IOException {
     final int volCount = 2000;
     final int step = 100;
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     for (int x = 0; x < volCount; x++) {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
-      OzoneVolume vol = client.createVolume(volumeName, "frodo", "100TB");
-      assertNotNull(vol);
+      client.createVolume(volumeName);
     }
-    OzoneVolume prevKey = null;
+    String prevKey = null;
     int count = 0;
     int pagecount = 0;
     while (count < volCount) {
-      List<OzoneVolume> ovols = client.listVolumes("frodo", null, step,
-          prevKey);
+      List<OzoneVolume> ovols = client.listVolumes(null, prevKey, step);
       count += ovols.size();
-      prevKey = ovols.get(ovols.size() - 1);
+      prevKey = ovols.get(ovols.size() - 1).getName();
       pagecount++;
     }
     assertEquals(volCount / step, pagecount);
@@ -267,30 +249,35 @@ public class TestVolume {
   @Ignore
   @Test
   public void testListAllVolumes() throws OzoneException, IOException {
-    runTestListAllVolumes(ozoneRestClient);
+    runTestListAllVolumes(client);
   }
 
-  static void runTestListAllVolumes(OzoneRestClient client)
+  static void runTestListAllVolumes(ClientProtocol client)
       throws OzoneException, IOException {
     final int volCount = 200;
     final int step = 10;
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     for (int x = 0; x < volCount; x++) {
       String userName =
           "frodo" + RandomStringUtils.randomAlphabetic(5).toLowerCase();
       String volumeName =
           "vol" + RandomStringUtils.randomAlphabetic(5).toLowerCase();
-      OzoneVolume vol = client.createVolume(volumeName, userName, "100TB");
+      VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+          .setOwner(userName)
+          .setQuota("100TB")
+          .setAdmin("hdfs")
+          .build();
+      client.createVolume(volumeName, volumeArgs);
+      OzoneVolume vol = client.getVolumeDetails(volumeName);
       assertNotNull(vol);
     }
-    OzoneVolume prevKey = null;
+    String prevKey = null;
     int count = 0;
     int pagecount = 0;
     while (count < volCount) {
-      List<OzoneVolume> ovols = client.listAllVolumes(null, step, prevKey);
+      List<OzoneVolume> ovols = client.listVolumes(null, prevKey, step);
       count += ovols.size();
       if (ovols.size() > 0) {
-        prevKey = ovols.get(ovols.size() - 1);
+        prevKey = ovols.get(ovols.size() - 1).getName();
       }
       pagecount++;
     }
@@ -301,17 +288,16 @@ public class TestVolume {
 
   @Test
   public void testListVolumes() throws Exception {
-    runTestListVolumes(ozoneRestClient);
+    runTestListVolumes(client);
   }
 
-  static void runTestListVolumes(OzoneRestClient client)
+  static void runTestListVolumes(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     final int volCount = 20;
     final String user1 = "test-user-a";
     final String user2 = "test-user-b";
 
     long currentTime = Time.now();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     // Create 20 volumes, 10 for user1 and another 10 for user2.
     for (int x = 0; x < volCount; x++) {
       String volumeName;
@@ -326,109 +312,40 @@ public class TestVolume {
         userName = user2;
         volumeName = "test-vol" + x;
       }
-      OzoneVolume vol = client.createVolume(volumeName, userName, "100TB");
+      VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+          .setOwner(userName)
+          .setQuota("100TB")
+          .setAdmin("hdfs")
+          .build();
+      client.createVolume(volumeName, volumeArgs);
+      OzoneVolume vol = client.getVolumeDetails(volumeName);
       assertNotNull(vol);
     }
 
     // list all the volumes belong to user1
-    List<OzoneVolume> volumeList = client.listVolumes(user1,
-        null, 100, StringUtils.EMPTY);
+    List<OzoneVolume> volumeList = client.listVolumes(user1, null, null, 100);
     assertEquals(10, volumeList.size());
     // verify the owner name and creation time of volume
     for (OzoneVolume vol : volumeList) {
-      assertTrue(vol.getOwnerName().equals(user1));
-      assertTrue((OzoneUtils.formatDate(vol.getCreatedOn())
+      assertTrue(vol.getOwner().equals(user1));
+      assertTrue((vol.getCreationTime()
           / 1000) >= (currentTime / 1000));
     }
 
     // test max key parameter of listing volumes
-    volumeList = client.listVolumes(user1, null, 2, StringUtils.EMPTY);
+    volumeList = client.listVolumes(user1, null, null, 2);
     assertEquals(2, volumeList.size());
 
     // test prefix parameter of listing volumes
-    volumeList = client.listVolumes(user1, "test-vol10", 100,
-        StringUtils.EMPTY);
+    volumeList = client.listVolumes(user1, "test-vol10", null, 10);
     assertTrue(volumeList.size() == 1
-        && volumeList.get(0).getVolumeName().equals("test-vol10"));
+        && volumeList.get(0).getName().equals("test-vol10"));
 
-    volumeList = client.listVolumes(user1, "test-vol1",
-        100, StringUtils.EMPTY);
+    volumeList = client.listVolumes(user1, "test-vol1", null, 10);
     assertEquals(5, volumeList.size());
 
     // test start key parameter of listing volumes
-    volumeList = client.listVolumes(user2, null, 100, "test-vol15");
+    volumeList = client.listVolumes(user2, null, "test-vol15", 10);
     assertEquals(2, volumeList.size());
   }
-
-  /**
-   * Returns a list of mocked {@link CloseableHttpClient} used for testing.
-   * The mocked client replaces the actual calls in
-   * {@link OzoneRestClient#newHttpClient()}, it is used to verify
-   * if the invocation of this client is expected. <b>Note</b>, the output
-   * of this method is always used as the input of
-   * {@link TestVolume#verifyHttpConnectionClosed(List)}.
-   *
-   * @param mockedClient mocked ozone client.
-   * @return a list of mocked {@link CloseableHttpClient}.
-   * @throws IOException
-   */
-  private static List<CloseableHttpClient> mockHttpClients(
-      OzoneRestClient mockedClient)
-      throws IOException {
-    List<CloseableHttpClient> spyHttpClients = new ArrayList<>();
-    for (int i = 0; i < 5; i++) {
-      CloseableHttpClient spyHttpClient = Mockito
-          .spy(HddsClientUtils.newHttpClient());
-      spyHttpClients.add(spyHttpClient);
-    }
-
-    List<CloseableHttpClient> nextReturns =
-        new ArrayList<>(spyHttpClients.subList(1, spyHttpClients.size()));
-    Mockito.when(mockedClient.newHttpClient()).thenReturn(
-        spyHttpClients.get(0),
-        nextReturns.toArray(new CloseableHttpClient[nextReturns.size()]));
-    return spyHttpClients;
-  }
-
-  /**
-   * This method is used together with
-   * {@link TestVolume#mockHttpClients(OzoneRestClient)} to verify
-   * if the http client is properly closed. It verifies that as long as
-   * a client calls {@link CloseableHttpClient#execute(HttpUriRequest)} to
-   * send request, then it must calls {@link CloseableHttpClient#close()}
-   * close the http connection.
-   *
-   * @param mockedHttpClients
-   */
-  private static void verifyHttpConnectionClosed(
-      List<CloseableHttpClient> mockedHttpClients) {
-    final AtomicInteger totalCalled = new AtomicInteger();
-    assertTrue(mockedHttpClients.stream().allMatch(closeableHttpClient -> {
-      boolean clientUsed = false;
-      try {
-        verify(closeableHttpClient, times(1)).execute(Mockito.any());
-        totalCalled.incrementAndGet();
-        clientUsed = true;
-      } catch (Throwable e) {
-        // There might be some redundant instances in mockedHttpClients,
-        // it is allowed that a client is not used.
-        return true;
-      }
-
-      if (clientUsed) {
-        try {
-          // If a client is used, ensure the close function is called.
-          verify(closeableHttpClient, times(1)).close();
-          return true;
-        } catch (IOException e) {
-          return false;
-        }
-      } else {
-        return true;
-      }
-    }));
-    System.out.println("Successful connections " + totalCalled.get());
-    assertTrue("The mocked http client should be called at least once.",
-        totalCalled.get() > 0);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
index 8314851..bc4ba25 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
@@ -23,7 +23,9 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 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.rpc.RpcClient;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -38,7 +40,7 @@ import java.io.IOException;
 public class TestVolumeRatis {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
-  private static OzoneRestClient ozoneClient;
+  private static ClientProtocol client;
   private static MiniOzoneCluster cluster;
 
   @BeforeClass
@@ -63,8 +65,7 @@ public class TestVolumeRatis {
     final int port = cluster.getHddsDatanodes().get(0)
         .getDatanodeDetails().getOzoneRestPort();
 
-    ozoneClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
+    client = new RpcClient(conf);
   }
 
   @AfterClass
@@ -77,53 +78,53 @@ public class TestVolumeRatis {
 
   @Test
   public void testCreateVolume() throws Exception {
-    TestVolume.runTestCreateVolume(ozoneClient);
+    TestVolume.runTestCreateVolume(client);
   }
 
   @Test
-  public void testCreateDuplicateVolume() throws OzoneException {
-    TestVolume.runTestCreateDuplicateVolume(ozoneClient);
+  public void testCreateDuplicateVolume() throws OzoneException, IOException {
+    TestVolume.runTestCreateDuplicateVolume(client);
   }
 
   @Test
-  public void testDeleteVolume() throws OzoneException {
-    TestVolume.runTestDeleteVolume(ozoneClient);
+  public void testDeleteVolume() throws OzoneException, IOException {
+    TestVolume.runTestDeleteVolume(client);
   }
 
   @Test
   public void testChangeOwnerOnVolume() throws Exception {
-    TestVolume.runTestChangeOwnerOnVolume(ozoneClient);
+    TestVolume.runTestChangeOwnerOnVolume(client);
   }
 
   @Test
   public void testChangeQuotaOnVolume() throws Exception {
-    TestVolume.runTestChangeQuotaOnVolume(ozoneClient);
+    TestVolume.runTestChangeQuotaOnVolume(client);
   }
 
   // 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);
+    TestVolume.runTestListVolume(client);
   }
 
   // TODO: remove @Ignore below once the problem has been resolved.
   @Ignore("See TestVolume.testListVolumePagination()")
   @Test
   public void testListVolumePagination() throws OzoneException, IOException {
-    TestVolume.runTestListVolumePagination(ozoneClient);
+    TestVolume.runTestListVolumePagination(client);
   }
 
   // TODO: remove @Ignore below once the problem has been resolved.
   @Ignore("See TestVolume.testListAllVolumes()")
   @Test
   public void testListAllVolumes() throws Exception {
-    TestVolume.runTestListAllVolumes(ozoneClient);
+    TestVolume.runTestListAllVolumes(client);
   }
 
   @Ignore("Disabling Ratis tests for pipeline work.")
   @Test
   public void testListVolumes() throws Exception {
-    TestVolume.runTestListVolumes(ozoneClient);
+    TestVolume.runTestListVolumes(client);
   }
 }


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