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 xy...@apache.org on 2017/08/08 20:36:36 UTC

[1/8] hadoop git commit: HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 b153dbb9b -> 43d38114e


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
new file mode 100644
index 0000000..837d1b7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
@@ -0,0 +1,368 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client.rpc;
+
+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.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+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.web.exceptions.OzoneException;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * This class is to test all the public facing APIs of Ozone Client.
+ */
+public class TestOzoneRpcClient {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneClient ozClient = null;
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true and
+   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   *
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
+        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
+    cluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
+    OzoneClientFactory.setConfiguration(conf);
+    ozClient = OzoneClientFactory.getRpcClient();
+  }
+
+  @Test
+  public void testCreateVolume()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
+    Assert.assertEquals(volumeName, volume.getVolumeName());
+  }
+
+  @Test
+  public void testCreateVolumeWithOwner()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName, "test");
+    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
+    Assert.assertEquals(volumeName, volume.getVolumeName());
+    Assert.assertEquals("test", volume.getOwnerName());
+  }
+
+  @Test
+  public void testCreateVolumeWithQuota()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName, "test",
+        10000000000L);
+    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
+    Assert.assertEquals(volumeName, volume.getVolumeName());
+    Assert.assertEquals("test", volume.getOwnerName());
+    Assert.assertEquals(10000000000L, volume.getQuota());
+  }
+
+  @Test
+  public void testVolumeAlreadyExist()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    try {
+      ozClient.createVolume(volumeName);
+    } catch (IOException ex) {
+      Assert.assertEquals(
+          "Volume creation failed, error:VOLUME_ALREADY_EXISTS",
+          ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testSetVolumeOwner()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.setVolumeOwner(volumeName, "test");
+    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
+    Assert.assertEquals("test", volume.getOwnerName());
+  }
+
+  @Test
+  public void testSetVolumeQuota()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.setVolumeQuota(volumeName, 10000000000L);
+    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
+    Assert.assertEquals(10000000000L, volume.getQuota());
+  }
+
+  @Test
+  public void testDeleteVolume()
+      throws IOException, OzoneException {
+    thrown.expectMessage("Info Volume failed, error");
+    String volumeName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
+    Assert.assertNotNull(volume);
+    ozClient.deleteVolume(volumeName);
+    ozClient.getVolumeDetails(volumeName);
+  }
+
+  @Test
+  public void testCreateBucket()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+  }
+
+  @Test
+  public void testCreateBucketWithVersioning()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName,
+        OzoneConsts.Versioning.ENABLED);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+    Assert.assertEquals(OzoneConsts.Versioning.ENABLED,
+        bucket.getVersioning());
+  }
+
+  @Test
+  public void testCreateBucketWithStorageType()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName, StorageType.SSD);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+    Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
+  }
+
+  @Test
+  public void testCreateBucketWithAcls()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
+        OzoneAcl.OzoneACLRights.READ_WRITE);
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName, userAcl);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+    Assert.assertTrue(bucket.getAcls().contains(userAcl));
+  }
+
+  @Test
+  public void testCreateBucketWithAllArgument()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
+        OzoneAcl.OzoneACLRights.READ_WRITE);
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName,
+        OzoneConsts.Versioning.ENABLED,
+        StorageType.SSD, userAcl);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+    Assert.assertEquals(OzoneConsts.Versioning.ENABLED,
+        bucket.getVersioning());
+    Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
+    Assert.assertTrue(bucket.getAcls().contains(userAcl));
+  }
+
+  @Test
+  public void testCreateBucketInInvalidVolume()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    try {
+      ozClient.createBucket(volumeName, bucketName);
+    } catch (IOException ex) {
+      Assert.assertEquals(
+          "Bucket creation failed, error: VOLUME_NOT_FOUND",
+          ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testAddBucketAcl()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName);
+    List<OzoneAcl> acls = new ArrayList<>();
+    acls.add(new OzoneAcl(
+        OzoneAcl.OzoneACLType.USER, "test",
+        OzoneAcl.OzoneACLRights.READ_WRITE));
+    ozClient.addBucketAcls(volumeName, bucketName, acls);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+    Assert.assertTrue(bucket.getAcls().contains(acls.get(0)));
+  }
+
+  @Test
+  public void testRemoveBucketAcl()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
+        OzoneAcl.OzoneACLRights.READ_WRITE);
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName, userAcl);
+    List<OzoneAcl> acls = new ArrayList<>();
+    acls.add(userAcl);
+    ozClient.removeBucketAcls(volumeName, bucketName, acls);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+    Assert.assertTrue(!bucket.getAcls().contains(acls.get(0)));
+  }
+
+  @Test
+  public void testSetBucketVersioning()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName);
+    ozClient.setBucketVersioning(volumeName, bucketName,
+        OzoneConsts.Versioning.ENABLED);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+    Assert.assertEquals(OzoneConsts.Versioning.ENABLED,
+        bucket.getVersioning());
+  }
+
+  @Test
+  public void testSetBucketStorageType()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName);
+    ozClient.setBucketStorageType(volumeName, bucketName,
+        StorageType.SSD);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertEquals(bucketName, bucket.getBucketName());
+    Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
+  }
+
+
+  @Test
+  public void testDeleteBucket()
+      throws IOException, OzoneException {
+    thrown.expectMessage("Info Bucket failed, error");
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName);
+    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
+    Assert.assertNotNull(bucket);
+    ozClient.deleteBucket(volumeName, bucketName);
+    ozClient.getBucketDetails(volumeName, bucketName);
+  }
+
+
+  @Test
+  public void testPutKey()
+      throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+    String value = "sample value";
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName);
+    OzoneOutputStream out = ozClient.createKey(volumeName, bucketName,
+        keyName, value.getBytes().length);
+    out.write(value.getBytes());
+    out.close();
+    OzoneKey key = ozClient.getKeyDetails(volumeName, bucketName, keyName);
+    Assert.assertEquals(keyName, key.getKeyName());
+    OzoneInputStream is = ozClient.getKey(volumeName, bucketName, keyName);
+    byte[] fileContent = new byte[value.getBytes().length];
+    is.read(fileContent);
+    Assert.assertEquals(value, new String(fileContent));
+  }
+
+  @Test
+  public void testDeleteKey()
+      throws IOException, OzoneException {
+    thrown.expectMessage("Lookup key failed, error");
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+    String value = "sample value";
+    ozClient.createVolume(volumeName);
+    ozClient.createBucket(volumeName, bucketName);
+    OzoneOutputStream out = ozClient.createKey(volumeName, bucketName,
+        keyName, value.getBytes().length);
+    out.write(value.getBytes());
+    out.close();
+    OzoneKey key = ozClient.getKeyDetails(volumeName, bucketName, keyName);
+    Assert.assertEquals(keyName, key.getKeyName());
+    ozClient.deleteKey(volumeName, bucketName, keyName);
+    ozClient.getKeyDetails(volumeName, bucketName, keyName);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/package-info.java
new file mode 100644
index 0000000..0f48495
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client.rpc;
+
+/**
+ * This package contains test class for Ozone rpc client library.
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 754d2af..6eafdaa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.scm.XceiverClient;
 import org.apache.hadoop.scm.XceiverClientSpi;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java
index a40a6d0..852c1ff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestBucketManagerImpl.java
@@ -18,8 +18,8 @@ package org.apache.hadoop.ozone.ksm;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
 import org.apache.hadoop.ozone.ksm.exceptions

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestChunkStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestChunkStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestChunkStreams.java
index 2c936df..568f1eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestChunkStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestChunkStreams.java
@@ -17,8 +17,8 @@
 package org.apache.hadoop.ozone.ksm;
 
 import org.apache.commons.lang.RandomStringUtils;
-import org.apache.hadoop.ozone.web.storage.ChunkGroupInputStream;
-import org.apache.hadoop.ozone.web.storage.ChunkGroupOutputStream;
+import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
+import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneHelper.java
index 791504e..5393f33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneHelper.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.ozone.web;
 
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.util.Time;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
index f7e54c7..09dd797 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.http.HttpResponse;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
index 33002ae..69e9dcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.Status;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;


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


[7/8] hadoop git commit: HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.

Posted by xy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
deleted file mode 100644
index 2ff6be9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
+++ /dev/null
@@ -1,651 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.protocolPB;
-
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtocolTranslator;
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
-import org.apache.hadoop.ksm.protocol.KeySpaceManagerProtocol;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.BucketArgs;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.BucketInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CreateBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CreateBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.InfoBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.InfoBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.SetBucketPropertyRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.SetBucketPropertyResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.DeleteBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.DeleteBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CreateVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CreateVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.LocateKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.LocateKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.KeyArgs;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.SetVolumePropertyRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.SetVolumePropertyResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.DeleteVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.DeleteVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.InfoVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.InfoVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CheckVolumeAccessRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CheckVolumeAccessResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ListBucketsRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ListBucketsResponse;
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.ListKeysRequest;
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.ListKeysResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.VolumeInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.Status;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
-    .ListVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
-    .ListVolumeResponse;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.stream.Collectors;
-
-/**
- *  The client side implementation of KeySpaceManagerProtocol.
- */
-
-@InterfaceAudience.Private
-public final class KeySpaceManagerProtocolClientSideTranslatorPB
-    implements KeySpaceManagerProtocol, ProtocolTranslator, Closeable {
-
-  /**
-   * RpcController is not used and hence is set to null.
-   */
-  private static final RpcController NULL_RPC_CONTROLLER = null;
-
-  private final KeySpaceManagerProtocolPB rpcProxy;
-
-  /**
-   * Constructor for KeySpaceManger Client.
-   * @param rpcProxy
-   */
-  public KeySpaceManagerProtocolClientSideTranslatorPB(
-      KeySpaceManagerProtocolPB rpcProxy) {
-    this.rpcProxy = rpcProxy;
-  }
-
-  /**
-   * Closes this stream and releases any system resources associated
-   * with it. If the stream is already closed then invoking this
-   * method has no effect.
-   * <p>
-   * <p> As noted in {@link AutoCloseable#close()}, cases where the
-   * close may fail require careful attention. It is strongly advised
-   * to relinquish the underlying resources and to internally
-   * <em>mark</em> the {@code Closeable} as closed, prior to throwing
-   * the {@code IOException}.
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void close() throws IOException {
-
-  }
-
-  /**
-   * Creates a volume.
-   *
-   * @param args - Arguments to create Volume.
-   * @throws IOException
-   */
-  @Override
-  public void createVolume(KsmVolumeArgs args) throws IOException {
-    CreateVolumeRequest.Builder req =
-        CreateVolumeRequest.newBuilder();
-    VolumeInfo volumeInfo = args.getProtobuf();
-    req.setVolumeInfo(volumeInfo);
-
-    final CreateVolumeResponse resp;
-    try {
-      resp = rpcProxy.createVolume(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Volume creation failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Changes the owner of a volume.
-   *
-   * @param volume - Name of the volume.
-   * @param owner - Name of the owner.
-   * @throws IOException
-   */
-  @Override
-  public void setOwner(String volume, String owner) throws IOException {
-    SetVolumePropertyRequest.Builder req =
-        SetVolumePropertyRequest.newBuilder();
-    req.setVolumeName(volume).setOwnerName(owner);
-    final SetVolumePropertyResponse resp;
-    try {
-      resp = rpcProxy.setVolumeProperty(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Volume owner change failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Changes the Quota on a volume.
-   *
-   * @param volume - Name of the volume.
-   * @param quota - Quota in bytes.
-   * @throws IOException
-   */
-  @Override
-  public void setQuota(String volume, long quota) throws IOException {
-    SetVolumePropertyRequest.Builder req =
-        SetVolumePropertyRequest.newBuilder();
-    req.setVolumeName(volume).setQuotaInBytes(quota);
-    final SetVolumePropertyResponse resp;
-    try {
-      resp = rpcProxy.setVolumeProperty(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Volume quota change failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Checks if the specified user can access this volume.
-   *
-   * @param volume - volume
-   * @param userAcl - user acls which needs to be checked for access
-   * @return true if the user has required access for the volume,
-   *         false otherwise
-   * @throws IOException
-   */
-  @Override
-  public boolean checkVolumeAccess(String volume, OzoneAclInfo userAcl) throws
-      IOException {
-    CheckVolumeAccessRequest.Builder req =
-        CheckVolumeAccessRequest.newBuilder();
-    req.setVolumeName(volume).setUserAcl(userAcl);
-    final CheckVolumeAccessResponse resp;
-    try {
-      resp = rpcProxy.checkVolumeAccess(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() == Status.ACCESS_DENIED) {
-      return false;
-    } else if (resp.getStatus() == Status.OK) {
-      return true;
-    } else {
-      throw new
-          IOException("Check Volume Access failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Gets the volume information.
-   *
-   * @param volume - Volume name.
-   * @return KsmVolumeArgs or exception is thrown.
-   * @throws IOException
-   */
-  @Override
-  public KsmVolumeArgs getVolumeInfo(String volume) throws IOException {
-    InfoVolumeRequest.Builder req = InfoVolumeRequest.newBuilder();
-    req.setVolumeName(volume);
-    final InfoVolumeResponse resp;
-    try {
-      resp = rpcProxy.infoVolume(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Info Volume failed, error:" + resp.getStatus());
-    }
-    return KsmVolumeArgs.getFromProtobuf(resp.getVolumeInfo());
-  }
-
-  /**
-   * Deletes an existing empty volume.
-   *
-   * @param volume - Name of the volume.
-   * @throws IOException
-   */
-  @Override
-  public void deleteVolume(String volume) throws IOException {
-    DeleteVolumeRequest.Builder req = DeleteVolumeRequest.newBuilder();
-    req.setVolumeName(volume);
-    final DeleteVolumeResponse resp;
-    try {
-      resp = rpcProxy.deleteVolume(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Delete Volume failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Lists volume owned by a specific user.
-   *
-   * @param userName - user name
-   * @param prefix - Filter prefix -- Return only entries that match this.
-   * @param prevKey - Previous key -- List starts from the next from the
-   * prevkey
-   * @param maxKeys - Max number of keys to return.
-   * @return List of Volumes.
-   * @throws IOException
-   */
-  @Override
-  public List<KsmVolumeArgs> listVolumeByUser(String userName, String prefix,
-                                              String prevKey, int maxKeys)
-      throws IOException {
-    ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
-    if (!Strings.isNullOrEmpty(prefix)) {
-      builder.setPrefix(prefix);
-    }
-    if (!Strings.isNullOrEmpty(prevKey)) {
-      builder.setPrevKey(prevKey);
-    }
-    builder.setMaxKeys(maxKeys);
-    builder.setUserName(userName);
-    builder.setScope(ListVolumeRequest.Scope.VOLUMES_BY_USER);
-    return listVolume(builder.build());
-  }
-
-  /**
-   * Lists volume all volumes in the cluster.
-   *
-   * @param prefix - Filter prefix -- Return only entries that match this.
-   * @param prevKey - Previous key -- List starts from the next from the
-   * prevkey
-   * @param maxKeys - Max number of keys to return.
-   * @return List of Volumes.
-   * @throws IOException
-   */
-  @Override
-  public List<KsmVolumeArgs> listAllVolumes(String prefix, String prevKey,
-      int maxKeys) throws IOException {
-    ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
-    if (!Strings.isNullOrEmpty(prefix)) {
-      builder.setPrefix(prefix);
-    }
-    if (!Strings.isNullOrEmpty(prevKey)) {
-      builder.setPrevKey(prevKey);
-    }
-    builder.setMaxKeys(maxKeys);
-    builder.setScope(ListVolumeRequest.Scope.VOLUMES_BY_CLUSTER);
-    return listVolume(builder.build());
-  }
-
-  private List<KsmVolumeArgs> listVolume(ListVolumeRequest request)
-      throws IOException {
-    final ListVolumeResponse resp;
-    try {
-      resp = rpcProxy.listVolumes(NULL_RPC_CONTROLLER, request);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("List volume failed, error: "
-          + resp.getStatus());
-    }
-
-    List<KsmVolumeArgs> result = Lists.newArrayList();
-    for (VolumeInfo volInfo : resp.getVolumeInfoList()) {
-      KsmVolumeArgs volArgs = KsmVolumeArgs.getFromProtobuf(volInfo);
-      result.add(volArgs);
-    }
-
-    return resp.getVolumeInfoList().stream()
-        .map(item -> KsmVolumeArgs.getFromProtobuf(item))
-        .collect(Collectors.toList());
-  }
-
-  /**
-   * Creates a bucket.
-   *
-   * @param bucketInfo - BucketInfo to create bucket.
-   * @throws IOException
-   */
-  @Override
-  public void createBucket(KsmBucketInfo bucketInfo) throws IOException {
-    CreateBucketRequest.Builder req =
-        CreateBucketRequest.newBuilder();
-    BucketInfo bucketInfoProtobuf = bucketInfo.getProtobuf();
-    req.setBucketInfo(bucketInfoProtobuf);
-
-    final CreateBucketResponse resp;
-    try {
-      resp = rpcProxy.createBucket(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Bucket creation failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * Gets the bucket information.
-   *
-   * @param volume - Volume name.
-   * @param bucket - Bucket name.
-   * @return KsmBucketInfo or exception is thrown.
-   * @throws IOException
-   */
-  @Override
-  public KsmBucketInfo getBucketInfo(String volume, String bucket)
-      throws IOException {
-    InfoBucketRequest.Builder req =
-        InfoBucketRequest.newBuilder();
-    req.setVolumeName(volume);
-    req.setBucketName(bucket);
-
-    final InfoBucketResponse resp;
-    try {
-      resp = rpcProxy.infoBucket(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() == Status.OK) {
-      return KsmBucketInfo.getFromProtobuf(resp.getBucketInfo());
-    } else {
-      throw new IOException("Info Bucket failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * Sets bucket property from args.
-   * @param args - BucketArgs.
-   * @throws IOException
-   */
-  @Override
-  public void setBucketProperty(KsmBucketArgs args)
-      throws IOException {
-    SetBucketPropertyRequest.Builder req =
-        SetBucketPropertyRequest.newBuilder();
-    BucketArgs bucketArgs = args.getProtobuf();
-    req.setBucketArgs(bucketArgs);
-    final SetBucketPropertyResponse resp;
-    try {
-      resp = rpcProxy.setBucketProperty(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Setting bucket property failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * List buckets in a volume.
-   *
-   * @param volumeName
-   * @param startKey
-   * @param prefix
-   * @param count
-   * @return
-   * @throws IOException
-   */
-  @Override
-  public List<KsmBucketInfo> listBuckets(String volumeName,
-      String startKey, String prefix, int count) throws IOException {
-    List<KsmBucketInfo> buckets = new ArrayList<>();
-    ListBucketsRequest.Builder reqBuilder = ListBucketsRequest.newBuilder();
-    reqBuilder.setVolumeName(volumeName);
-    reqBuilder.setCount(count);
-    if (startKey != null) {
-      reqBuilder.setStartKey(startKey);
-    }
-    if (prefix != null) {
-      reqBuilder.setPrefix(prefix);
-    }
-    ListBucketsRequest request = reqBuilder.build();
-    final ListBucketsResponse resp;
-    try {
-      resp = rpcProxy.listBuckets(NULL_RPC_CONTROLLER, request);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() == Status.OK) {
-      buckets.addAll(
-          resp.getBucketInfoList().stream()
-              .map(KsmBucketInfo::getFromProtobuf)
-              .collect(Collectors.toList()));
-      return buckets;
-    } else {
-      throw new IOException("List Buckets failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * Allocate a block for a key, then use the returned meta info to talk to data
-   * node to actually write the key.
-   * @param args the args for the key to be allocated
-   * @return a handler to the key, returned client
-   * @throws IOException
-   */
-  @Override
-  public KsmKeyInfo allocateKey(KsmKeyArgs args) throws IOException {
-    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setKeyName(args.getKeyName())
-        .setDataSize(args.getDataSize()).build();
-    req.setKeyArgs(keyArgs);
-
-    final LocateKeyResponse resp;
-    try {
-      resp = rpcProxy.createKey(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Create key failed, error:" +
-          resp.getStatus());
-    }
-    return KsmKeyInfo.getFromProtobuf(resp.getKeyInfo());
-  }
-
-  @Override
-  public KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException {
-    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setKeyName(args.getKeyName())
-        .setDataSize(args.getDataSize()).build();
-    req.setKeyArgs(keyArgs);
-
-    final LocateKeyResponse resp;
-    try {
-      resp = rpcProxy.lookupKey(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Lookup key failed, error:" +
-          resp.getStatus());
-    }
-    return KsmKeyInfo.getFromProtobuf(resp.getKeyInfo());
-  }
-
-  /**
-   * Deletes an existing key.
-   *
-   * @param args the args of the key.
-   * @throws IOException
-   */
-  @Override
-  public void deleteKey(KsmKeyArgs args) throws IOException {
-    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setKeyName(args.getKeyName()).build();
-    req.setKeyArgs(keyArgs);
-
-    final LocateKeyResponse resp;
-    try {
-      resp = rpcProxy.deleteKey(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Delete key failed, error:" +
-          resp.getStatus());
-    }
-  }
-
-  /**
-   * Deletes an existing empty bucket from volume.
-   * @param volume - Name of the volume.
-   * @param bucket - Name of the bucket.
-   * @throws IOException
-   */
-  public void deleteBucket(String volume, String bucket) throws IOException {
-    DeleteBucketRequest.Builder req = DeleteBucketRequest.newBuilder();
-    req.setVolumeName(volume);
-    req.setBucketName(bucket);
-    final DeleteBucketResponse resp;
-    try {
-      resp = rpcProxy.deleteBucket(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Delete Bucket failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * List keys in a bucket.
-   */
-  @Override
-  public List<KsmKeyInfo> listKeys(String volumeName, String bucketName,
-      String startKey, String prefix, int maxKeys) throws IOException {
-    List<KsmKeyInfo> keys = new ArrayList<>();
-    ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder();
-    reqBuilder.setVolumeName(volumeName);
-    reqBuilder.setBucketName(bucketName);
-    reqBuilder.setCount(maxKeys);
-
-    if (startKey != null) {
-      reqBuilder.setStartKey(startKey);
-    }
-
-    if (prefix != null) {
-      reqBuilder.setPrefix(prefix);
-    }
-
-    ListKeysRequest request = reqBuilder.build();
-    final ListKeysResponse resp;
-    try {
-      resp = rpcProxy.listKeys(NULL_RPC_CONTROLLER, request);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() == Status.OK) {
-      keys.addAll(
-          resp.getKeyInfoList().stream()
-              .map(KsmKeyInfo::getFromProtobuf)
-              .collect(Collectors.toList()));
-      return keys;
-    } else {
-      throw new IOException("List Keys failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * Return the proxy object underlying this protocol translator.
-   *
-   * @return the proxy object underlying this protocol translator.
-   */
-  @Override
-  public Object getUnderlyingProxyObject() {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/KeySpaceManagerProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/KeySpaceManagerProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/KeySpaceManagerProtocolPB.java
deleted file mode 100644
index 8b960a9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/KeySpaceManagerProtocolPB.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.protocolPB;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.KeySpaceManagerService;
-
-/**
- * Protocol used to communicate with KSM.
- */
-@ProtocolInfo(protocolName =
-    "org.apache.hadoop.ozone.protocol.KeySpaceManagerProtocol",
-    protocolVersion = 1)
-@InterfaceAudience.Private
-public interface KeySpaceManagerProtocolPB
-    extends KeySpaceManagerService.BlockingInterface {
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/package-info.java
deleted file mode 100644
index 5dfa5ea..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocolPB/package-info.java
+++ /dev/null
@@ -1,19 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ksm.protocolPB;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
index 0aca0ad..d712074 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
@@ -84,6 +84,11 @@ public class OzoneAcl {
     return new OzoneAcl(aclType, parts[1], rights);
   }
 
+  @Override
+  public String toString() {
+    return type+":" + name + ":" + rights;
+  }
+
   /**
    * Returns a hash code value for the object. This method is
    * supported for the benefit of hash tables.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
new file mode 100644
index 0000000..55b5f88
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.scm.ScmConfigKeys;
+
+/**
+ * This class contains constants for configuration keys used in Ozone.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class OzoneConfigKeys {
+  public static final String DFS_CONTAINER_IPC_PORT =
+      "dfs.container.ipc";
+  public static final int DFS_CONTAINER_IPC_PORT_DEFAULT = 50011;
+
+  /**
+   *
+   * When set to true, allocate a random free port for ozone container,
+   * so that a mini cluster is able to launch multiple containers on a node.
+   *
+   * When set to false (default), container port is fixed as specified by
+   * DFS_CONTAINER_IPC_PORT_DEFAULT.
+   */
+  public static final String DFS_CONTAINER_IPC_RANDOM_PORT =
+      "dfs.container.ipc.random.port";
+  public static final boolean DFS_CONTAINER_IPC_RANDOM_PORT_DEFAULT =
+      false;
+
+  public static final String OZONE_LOCALSTORAGE_ROOT =
+      "ozone.localstorage.root";
+  public static final String OZONE_LOCALSTORAGE_ROOT_DEFAULT = "/tmp/ozone";
+  public static final String OZONE_ENABLED =
+      "ozone.enabled";
+  public static final boolean OZONE_ENABLED_DEFAULT = false;
+  public static final String OZONE_HANDLER_TYPE_KEY =
+      "ozone.handler.type";
+  public static final String OZONE_HANDLER_TYPE_DEFAULT = "distributed";
+  public static final String OZONE_TRACE_ENABLED_KEY =
+      "ozone.trace.enabled";
+  public static final boolean OZONE_TRACE_ENABLED_DEFAULT = false;
+
+  public static final String OZONE_CONTAINER_METADATA_DIRS =
+      "ozone.container.metadata.dirs";
+
+  public static final String OZONE_METADATA_STORE_IMPL =
+      "ozone.metastore.impl";
+  public static final String OZONE_METADATA_STORE_IMPL_LEVELDB =
+      "LevelDB";
+  public static final String OZONE_METADATA_STORE_IMPL_ROCKSDB =
+      "RocksDB";
+  public static final String OZONE_METADATA_STORE_IMPL_DEFAULT =
+      OZONE_METADATA_STORE_IMPL_LEVELDB;
+
+  public static final String OZONE_KEY_CACHE = "ozone.key.cache.size";
+  public static final int OZONE_KEY_CACHE_DEFAULT = 1024;
+
+  public static final String OZONE_SCM_BLOCK_SIZE_KEY =
+      "ozone.scm.block.size";
+  public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256 * OzoneConsts.MB;
+
+  /**
+   * Ozone administrator users delimited by comma.
+   * If not set, only the user who launches an ozone service will be the
+   * admin user. This property must be set if ozone services are started by
+   * different users. Otherwise the RPC layer will reject calls from
+   * other servers which are started by users not in the list.
+   * */
+  public static final String OZONE_ADMINISTRATORS =
+      "ozone.administrators";
+
+  public static final String OZONE_CLIENT_SOCKET_TIMEOUT_MS =
+      "ozone.client.socket.timeout.ms";
+  public static final int OZONE_CLIENT_SOCKET_TIMEOUT_MS_DEFAULT = 5000;
+  public static final String OZONE_CLIENT_CONNECTION_TIMEOUT_MS =
+      "ozone.client.connection.timeout.ms";
+  public static final int OZONE_CLIENT_CONNECTION_TIMEOUT_MS_DEFAULT = 5000;
+
+  public static final String DFS_CONTAINER_RATIS_ENABLED_KEY
+      = ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
+  public static final boolean DFS_CONTAINER_RATIS_ENABLED_DEFAULT
+      = ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
+  public static final String DFS_CONTAINER_RATIS_RPC_TYPE_KEY
+      = ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY;
+  public static final String DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT
+      = ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT;
+  /** A unique ID to identify a Ratis server. */
+  public static final String DFS_CONTAINER_RATIS_SERVER_ID =
+      "dfs.container.ratis.server.id";
+  public static final String DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR =
+      "dfs.container.ratis.datanode.storage.dir";
+
+  public static final String OZONE_SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL =
+      "ozone.web.authentication.kerberos.principal";
+
+  /**
+   * There is no need to instantiate this class.
+   */
+  private OzoneConfigKeys() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
new file mode 100644
index 0000000..a16e8d9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Configuration for ozone.
+ */
+@InterfaceAudience.Private
+public class OzoneConfiguration extends Configuration {
+  static {
+    // adds the default resources
+    Configuration.addDefaultResource("hdfs-default.xml");
+    Configuration.addDefaultResource("hdfs-site.xml");
+    Configuration.addDefaultResource("ozone-default.xml");
+    Configuration.addDefaultResource("ozone-site.xml");
+  }
+
+  public OzoneConfiguration() {
+  }
+
+  public OzoneConfiguration(Configuration conf) {
+    super(conf);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
new file mode 100644
index 0000000..bfd5714
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConsts.Versioning;
+
+import java.util.List;
+
+/**
+ * A class that encapsulates OzoneBucket.
+ */
+public class OzoneBucket {
+
+  /**
+   * Name of the volume in which the bucket belongs to.
+   */
+  private final String volumeName;
+  /**
+   * Name of the bucket.
+   */
+  private final String bucketName;
+  /**
+   * Bucket ACLs.
+   */
+  private final List<OzoneAcl> acls;
+
+  /**
+   * Type of storage to be used for this bucket.
+   * [RAM_DISK, SSD, DISK, ARCHIVE]
+   */
+  private final StorageType storageType;
+
+  /**
+   * Bucket Version flag.
+   */
+  private final Versioning versioning;
+
+
+  /**
+   * Constructs OzoneBucket from KsmBucketInfo.
+   *
+   * @param ksmBucketInfo
+   */
+  public OzoneBucket(KsmBucketInfo ksmBucketInfo) {
+    this.volumeName = ksmBucketInfo.getVolumeName();
+    this.bucketName = ksmBucketInfo.getBucketName();
+    this.acls = ksmBucketInfo.getAcls();
+    this.storageType = ksmBucketInfo.getStorageType();
+    this.versioning = ksmBucketInfo.getIsVersionEnabled() ?
+        Versioning.ENABLED : Versioning.DISABLED;
+  }
+
+  /**
+   * Returns Volume Name.
+   *
+   * @return volumeName
+   */
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  /**
+   * Returns Bucket Name.
+   *
+   * @return bucketName
+   */
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  /**
+   * Returns ACL's associated with the Bucket.
+   *
+   * @return acls
+   */
+  public List<OzoneAcl> getAcls() {
+    return acls;
+  }
+
+  /**
+   * Returns StorageType of the Bucket.
+   *
+   * @return storageType
+   */
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Returns Versioning associated with the Bucket.
+   *
+   * @return versioning
+   */
+  public Versioning getVersioning() {
+    return versioning;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java
new file mode 100644
index 0000000..a7808d8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java
@@ -0,0 +1,415 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConsts.Versioning;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * OzoneClient can connect to a Ozone Cluster and
+ * perform basic operations.
+ */
+public interface OzoneClient {
+
+  /**
+   * Creates a new Volume.
+   *
+   * @param volumeName Name of the Volume
+   *
+   * @throws IOException
+   */
+  void createVolume(String volumeName)
+      throws IOException;
+
+  /**
+   * Creates a new Volume, with owner set.
+   *
+   * @param volumeName Name of the Volume
+   * @param owner Owner to be set for Volume
+   *
+   * @throws IOException
+   */
+  void createVolume(String volumeName, String owner)
+      throws IOException;
+
+  /**
+   * Creates a new Volume, with owner and quota set.
+   *
+   * @param volumeName Name of the Volume
+   * @param owner Owner to be set for Volume
+   * @param acls ACLs to be added to the Volume
+   *
+   * @throws IOException
+   */
+  void createVolume(String volumeName, String owner,
+                    OzoneAcl... acls)
+      throws IOException;
+
+  /**
+   * Creates a new Volume, with owner and quota set.
+   *
+   * @param volumeName Name of the Volume
+   * @param owner Owner to be set for Volume
+   * @param quota Volume Quota
+   *
+   * @throws IOException
+   */
+  void createVolume(String volumeName, String owner,
+                    long quota)
+      throws IOException;
+
+  /**
+   * Creates a new Volume, with owner and quota set.
+   *
+   * @param volumeName Name of the Volume
+   * @param owner Owner to be set for Volume
+   * @param quota Volume Quota
+   * @param acls ACLs to be added to the Volume
+   *
+   * @throws IOException
+   */
+  void createVolume(String volumeName, String owner,
+                    long quota, OzoneAcl... acls)
+      throws IOException;
+
+  /**
+   * Sets the owner of the volume.
+   *
+   * @param volumeName Name of the Volume
+   * @param owner to be set for the Volume
+   *
+   * @throws IOException
+   */
+  void setVolumeOwner(String volumeName, String owner) throws IOException;
+
+  /**
+   * Set Volume Quota.
+   *
+   * @param volumeName Name of the Volume
+   * @param quota Quota to be set for the Volume
+   *
+   * @throws IOException
+   */
+  void setVolumeQuota(String volumeName, long quota)
+      throws IOException;
+
+  /**
+   * Returns {@link OzoneVolume}.
+   *
+   * @param volumeName Name of the Volume
+   *
+   * @return KsmVolumeArgs
+   *
+   * @throws OzoneVolume
+   * */
+  OzoneVolume getVolumeDetails(String volumeName)
+      throws IOException;
+
+  /**
+   * Checks if a Volume exists and the user with a role specified has access
+   * to the Volume.
+   *
+   * @param volumeName Name of the Volume
+   * @param acl requested acls which needs to be checked for access
+   *
+   * @return Boolean - True if the user with a role can access the volume.
+   * This is possible for owners of the volume and admin users
+   *
+   * @throws IOException
+   */
+  boolean checkVolumeAccess(String volumeName, OzoneAcl acl)
+      throws IOException;
+
+  /**
+   * Deletes an Empty Volume.
+   *
+   * @param volumeName Name of the Volume
+   *
+   * @throws IOException
+   */
+  void deleteVolume(String volumeName) throws IOException;
+
+  /**
+   * Returns the List of Volumes owned by current user.
+   *
+   * @param volumePrefix Volume prefix to match
+   *
+   * @return KsmVolumeArgs Iterator
+   *
+   * @throws IOException
+   */
+  Iterator<OzoneVolume> listVolumes(String volumePrefix)
+      throws IOException;
+
+  /**
+   * Returns the List of Volumes owned by the specific user.
+   *
+   * @param volumePrefix Volume prefix to match
+   * @param user User Name
+   *
+   * @return KsmVolumeArgs Iterator
+   *
+   * @throws IOException
+   */
+  Iterator<OzoneVolume> listVolumes(String volumePrefix, String user)
+      throws IOException;
+
+  /**
+   * Creates a new Bucket in the Volume.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   *
+   * @throws IOException
+   */
+  void createBucket(String volumeName, String bucketName)
+      throws IOException;
+
+  /**
+   * Creates a new Bucket in the Volume, with versioning set.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param versioning Bucket versioning
+   *
+   * @throws IOException
+   */
+  void createBucket(String volumeName, String bucketName,
+                    Versioning versioning)
+      throws IOException;
+
+  /**
+   * Creates a new Bucket in the Volume, with storage type set.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param storageType StorageType for the Bucket
+   *
+   * @throws IOException
+   */
+  void createBucket(String volumeName, String bucketName,
+                    StorageType storageType)
+      throws IOException;
+
+  /**
+   * Creates a new Bucket in the Volume, with ACLs set.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param acls OzoneAcls for the Bucket
+   *
+   * @throws IOException
+   */
+  void createBucket(String volumeName, String bucketName,
+                           OzoneAcl... acls)
+      throws IOException;
+
+
+  /**
+   * Creates a new Bucket in the Volume, with versioning
+   * storage type and ACLs set.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param storageType StorageType for the Bucket
+   *
+   * @throws IOException
+   */
+  void createBucket(String volumeName, String bucketName,
+                           Versioning versioning,
+                           StorageType storageType, OzoneAcl... acls)
+      throws IOException;
+
+  /**
+   * Adds or Removes ACLs from a Bucket.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   *
+   * @throws IOException
+   */
+  void addBucketAcls(String volumeName, String bucketName,
+                     List<OzoneAcl> addAcls)
+      throws IOException;
+
+  /**
+   * Adds or Removes ACLs from a Bucket.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   *
+   * @throws IOException
+   */
+  void removeBucketAcls(String volumeName, String bucketName,
+                        List<OzoneAcl> removeAcls)
+      throws IOException;
+
+
+  /**
+   * Enables or disables Bucket Versioning.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   *
+   * @throws IOException
+   */
+  void setBucketVersioning(String volumeName, String bucketName,
+                           Versioning versioning)
+      throws IOException;
+
+  /**
+   * Sets the Storage Class of a Bucket.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   *
+   * @throws IOException
+   */
+  void setBucketStorageType(String volumeName, String bucketName,
+                            StorageType storageType)
+      throws IOException;
+
+  /**
+   * Deletes a bucket if it is empty.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   *
+   * @throws IOException
+   */
+  void deleteBucket(String volumeName, String bucketName)
+      throws IOException;
+
+  /**
+   * true if the bucket exists and user has read access
+   * to the bucket else throws Exception.
+   *
+   * @param volumeName Name of the Volume
+   *
+   * @throws IOException
+   */
+  void checkBucketAccess(String volumeName, String bucketName)
+      throws IOException;
+
+    /**
+     * Returns {@link OzoneBucket}.
+     *
+     * @param volumeName Name of the Volume
+     * @param bucketName Name of the Bucket
+     *
+     * @return OzoneBucket
+     *
+     * @throws IOException
+     */
+  OzoneBucket getBucketDetails(String volumeName, String bucketName)
+        throws IOException;
+
+  /**
+   * Returns the List of Buckets in the Volume.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketPrefix Bucket prefix to match
+   *
+   * @return KsmVolumeArgs Iterator
+   *
+   * @throws IOException
+   */
+  Iterator<OzoneBucket> listBuckets(String volumeName, String bucketPrefix)
+      throws IOException;
+
+  /**
+   * Writes a key in an existing bucket.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param size Size of the data
+   *
+   * @return OutputStream
+   *
+   */
+  OzoneOutputStream createKey(String volumeName, String bucketName,
+                              String keyName, long size)
+      throws IOException;
+
+  /**
+   * Reads a key from an existing bucket.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   *
+   * @return LengthInputStream
+   *
+   * @throws IOException
+   */
+  OzoneInputStream getKey(String volumeName, String bucketName, String keyName)
+      throws IOException;
+
+
+  /**
+   * Deletes an existing key.
+   *
+   * @param volumeName Name of the Volume
+   *
+   * @throws IOException
+   */
+  void deleteKey(String volumeName, String bucketName, String keyName)
+      throws IOException;
+
+
+  /**
+   * Returns list of {@link OzoneKey} in Volume/Bucket.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   *
+   * @return OzoneKey
+   *
+   * @throws IOException
+   */
+  List<OzoneKey> listKeys(String volumeName, String bucketName,
+                            String keyPrefix)
+      throws IOException;
+
+
+  /**
+   * Get OzoneKey.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param keyName Key name
+   *
+   * @return OzoneKey
+   *
+   * @throws IOException
+   */
+  OzoneKey getKeyDetails(String volumeName, String bucketName,
+                        String keyName)
+      throws IOException;
+
+  /**
+   * Close and release the resources.
+   */
+  void close() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java
new file mode 100644
index 0000000..b098be9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.client.rest.OzoneRestClient;
+import org.apache.hadoop.ozone.client.rpc.OzoneRpcClient;
+
+import java.io.IOException;
+
+/**
+ * Factory class to create different types of OzoneClients.
+ */
+public final class OzoneClientFactory {
+
+  /**
+   * Private constructor, class is not meant to be initialized.
+   */
+  private OzoneClientFactory(){}
+
+  private static Configuration configuration;
+
+  /**
+   * Returns an OzoneClient which will use RPC protocol to perform
+   * client operations.
+   *
+   * @return OzoneClient
+   * @throws IOException
+   */
+  public static OzoneClient getClient() throws IOException {
+    //TODO: get client based on ozone.client.protocol
+    return new OzoneRpcClient(getConfiguration());
+  }
+
+  /**
+   * Returns an OzoneClient which will use RPC protocol to perform
+   * client operations.
+   *
+   * @return OzoneClient
+   * @throws IOException
+   */
+  public static OzoneClient getRpcClient() throws IOException {
+    return new OzoneRpcClient(getConfiguration());
+  }
+
+  /**
+   * Returns an OzoneClient which will use RPC protocol to perform
+   * client operations.
+   *
+   * @return OzoneClient
+   * @throws IOException
+   */
+  public static OzoneClient getRestClient() throws IOException {
+    return new OzoneRestClient(getConfiguration());
+  }
+
+  /**
+   * Sets the configuration, which will be used while creating OzoneClient.
+   *
+   * @param conf
+   */
+  public static void setConfiguration(Configuration conf) {
+    configuration = conf;
+  }
+
+  /**
+   * Returns the configuration if it's already set, else creates a new
+   * {@link OzoneConfiguration} and returns it.
+   *
+   * @return Configuration
+   */
+  private static synchronized Configuration getConfiguration() {
+    if(configuration == null) {
+      setConfiguration(new OzoneConfiguration());
+    }
+    return configuration;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
new file mode 100644
index 0000000..9390a85
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
@@ -0,0 +1,707 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+import com.google.common.base.Optional;
+
+import com.google.common.net.HostAndPort;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.scm.ScmConfigKeys;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.cblock.CBlockConfigKeys
+    .DFS_CBLOCK_SERVICERPC_ADDRESS_KEY;
+import static org.apache.hadoop.cblock.CBlockConfigKeys
+    .DFS_CBLOCK_SERVICERPC_HOSTNAME_DEFAULT;
+import static org.apache.hadoop.cblock.CBlockConfigKeys
+    .DFS_CBLOCK_SERVICERPC_PORT_DEFAULT;
+import static org.apache.hadoop.cblock.CBlockConfigKeys
+    .DFS_CBLOCK_JSCSIRPC_ADDRESS_KEY;
+import static org.apache.hadoop.cblock.CBlockConfigKeys
+    .DFS_CBLOCK_JSCSI_PORT_DEFAULT;
+
+import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY;
+import static org.apache.hadoop.ozone.ksm.KSMConfigKeys
+    .OZONE_KSM_BIND_HOST_DEFAULT;
+import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_PORT_DEFAULT;
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_DEADNODE_INTERVAL_DEFAULT;
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_DEADNODE_INTERVAL_MS;
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS;
+
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT;
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT;
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS;
+
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_STALENODE_INTERVAL_DEFAULT;
+import static org.apache.hadoop.scm.ScmConfigKeys
+    .OZONE_SCM_STALENODE_INTERVAL_MS;
+
+/**
+ * Utility methods for Ozone and Container Clients.
+ *
+ * The methods to retrieve SCM service endpoints assume there is a single
+ * SCM service instance. This will change when we switch to replicated service
+ * instances for redundancy.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class OzoneClientUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      OzoneClientUtils.class);
+  private static final int NO_PORT = -1;
+
+  /**
+   * The service ID of the solitary Ozone SCM service.
+   */
+  public static final String OZONE_SCM_SERVICE_ID = "OzoneScmService";
+  public static final String OZONE_SCM_SERVICE_INSTANCE_ID =
+      "OzoneScmServiceInstance";
+
+  private OzoneClientUtils() {
+    // Never constructed
+  }
+
+  /**
+   * Retrieve the socket addresses of all storage container managers.
+   *
+   * @param conf
+   * @return A collection of SCM addresses
+   * @throws IllegalArgumentException If the configuration is invalid
+   */
+  public static Collection<InetSocketAddress> getSCMAddresses(
+      Configuration conf) throws IllegalArgumentException {
+    Collection<InetSocketAddress> addresses =
+        new HashSet<InetSocketAddress>();
+    Collection<String> names =
+        conf.getTrimmedStringCollection(ScmConfigKeys.OZONE_SCM_NAMES);
+    if (names == null || names.isEmpty()) {
+      throw new IllegalArgumentException(ScmConfigKeys.OZONE_SCM_NAMES
+          + " need to be a set of valid DNS names or IP addresses."
+          + " Null or empty address list found.");
+    }
+
+    final com.google.common.base.Optional<Integer>
+        defaultPort =  com.google.common.base.Optional.of(ScmConfigKeys
+        .OZONE_SCM_DEFAULT_PORT);
+    for (String address : names) {
+      com.google.common.base.Optional<String> hostname =
+          OzoneClientUtils.getHostName(address);
+      if (!hostname.isPresent()) {
+        throw new IllegalArgumentException("Invalid hostname for SCM: "
+            + hostname);
+      }
+      com.google.common.base.Optional<Integer> port =
+          OzoneClientUtils.getHostPort(address);
+      InetSocketAddress addr = NetUtils.createSocketAddr(hostname.get(),
+          port.or(defaultPort.get()));
+      addresses.add(addr);
+    }
+    return addresses;
+  }
+
+  /**
+   * Retrieve the socket address that should be used by clients to connect
+   * to the SCM.
+   *
+   * @param conf
+   * @return Target InetSocketAddress for the SCM client endpoint.
+   */
+  public static InetSocketAddress getScmAddressForClients(Configuration conf) {
+    final Optional<String> host = getHostNameFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
+
+    if (!host.isPresent()) {
+      throw new IllegalArgumentException(
+          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY +
+          " must be defined. See" +
+          " https://wiki.apache.org/hadoop/Ozone#Configuration for details" +
+          " on configuring Ozone.");
+    }
+
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
+
+    return NetUtils.createSocketAddr(host.get() + ":" +
+        port.or(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
+  }
+
+  /**
+   * Retrieve the socket address that should be used by clients to connect
+   * to the SCM for block service. If
+   * {@link ScmConfigKeys#OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY} is not defined
+   * then {@link ScmConfigKeys#OZONE_SCM_CLIENT_ADDRESS_KEY} is used.
+   *
+   * @param conf
+   * @return Target InetSocketAddress for the SCM block client endpoint.
+   * @throws IllegalArgumentException if configuration is not defined.
+   */
+  public static InetSocketAddress getScmAddressForBlockClients(
+      Configuration conf) {
+    Optional<String> host = getHostNameFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY);
+
+    if (!host.isPresent()) {
+      host = getHostNameFromConfigKeys(conf,
+              ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
+      if (!host.isPresent()) {
+        throw new IllegalArgumentException(
+                ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY +
+                        " must be defined. See" +
+                        " https://wiki.apache.org/hadoop/Ozone#Configuration" +
+                        " for details on configuring Ozone.");
+      }
+    }
+
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY);
+
+    return NetUtils.createSocketAddr(host.get() + ":" +
+        port.or(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT));
+  }
+
+  /**
+   * Retrieve the socket address that should be used by DataNodes to connect
+   * to the SCM.
+   *
+   * @param conf
+   * @return Target InetSocketAddress for the SCM service endpoint.
+   */
+  public static InetSocketAddress getScmAddressForDataNodes(
+      Configuration conf) {
+    // We try the following settings in decreasing priority to retrieve the
+    // target host.
+    // - OZONE_SCM_DATANODE_ADDRESS_KEY
+    // - OZONE_SCM_CLIENT_ADDRESS_KEY
+    //
+    final Optional<String> host = getHostNameFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY,
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
+
+    if (!host.isPresent()) {
+      throw new IllegalArgumentException(
+          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY +
+          " must be defined. See" +
+          " https://wiki.apache.org/hadoop/Ozone#Configuration for details" +
+          " on configuring Ozone.");
+    }
+
+    // If no port number is specified then we'll just try the defaultBindPort.
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY);
+
+    InetSocketAddress addr = NetUtils.createSocketAddr(host.get() + ":" +
+        port.or(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
+
+    return addr;
+  }
+
+  /**
+   * Retrieve the socket address that should be used by clients to connect
+   * to the SCM.
+   *
+   * @param conf
+   * @return Target InetSocketAddress for the SCM client endpoint.
+   */
+  public static InetSocketAddress getScmClientBindAddress(
+      Configuration conf) {
+    final Optional<String> host = getHostNameFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY);
+
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
+
+    return NetUtils.createSocketAddr(
+        host.or(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_DEFAULT) + ":" +
+            port.or(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
+  }
+
+  /**
+   * Retrieve the socket address that should be used by clients to connect
+   * to the SCM Block service.
+   *
+   * @param conf
+   * @return Target InetSocketAddress for the SCM block client endpoint.
+   */
+  public static InetSocketAddress getScmBlockClientBindAddress(
+      Configuration conf) {
+    final Optional<String> host = getHostNameFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_BIND_HOST_KEY);
+
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY);
+
+    return NetUtils.createSocketAddr(
+        host.or(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_BIND_HOST_DEFAULT) +
+            ":" + port.or(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT));
+  }
+
+  /**
+   * Retrieve the socket address that should be used by DataNodes to connect
+   * to the SCM.
+   *
+   * @param conf
+   * @return Target InetSocketAddress for the SCM service endpoint.
+   */
+  public static InetSocketAddress getScmDataNodeBindAddress(
+      Configuration conf) {
+    final Optional<String> host = getHostNameFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY);
+
+    // If no port number is specified then we'll just try the defaultBindPort.
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY);
+
+    return NetUtils.createSocketAddr(
+        host.or(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_DEFAULT) + ":" +
+            port.or(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
+  }
+
+
+  /**
+   * Retrieve the socket address that is used by KSM.
+   * @param conf
+   * @return Target InetSocketAddress for the SCM service endpoint.
+   */
+  public static InetSocketAddress getKsmAddress(
+      Configuration conf) {
+    final Optional<String> host = getHostNameFromConfigKeys(conf,
+        OZONE_KSM_ADDRESS_KEY);
+
+    // If no port number is specified then we'll just try the defaultBindPort.
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        OZONE_KSM_ADDRESS_KEY);
+
+    return NetUtils.createSocketAddr(
+        host.or(OZONE_KSM_BIND_HOST_DEFAULT) + ":" +
+            port.or(OZONE_KSM_PORT_DEFAULT));
+  }
+
+  /**
+   * Retrieve the socket address that is used by CBlock Service.
+   * @param conf
+   * @return Target InetSocketAddress for the CBlock Service endpoint.
+   */
+  public static InetSocketAddress getCblockServiceRpcAddr(
+      Configuration conf) {
+    final Optional<String> host = getHostNameFromConfigKeys(conf,
+        DFS_CBLOCK_SERVICERPC_ADDRESS_KEY);
+
+    // If no port number is specified then we'll just try the defaultBindPort.
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        DFS_CBLOCK_SERVICERPC_ADDRESS_KEY);
+
+    return NetUtils.createSocketAddr(
+        host.or(DFS_CBLOCK_SERVICERPC_HOSTNAME_DEFAULT) + ":" +
+            port.or(DFS_CBLOCK_SERVICERPC_PORT_DEFAULT));
+  }
+
+  /**
+   * Retrieve the socket address that is used by CBlock Server.
+   * @param conf
+   * @return Target InetSocketAddress for the CBlock Server endpoint.
+   */
+  public static InetSocketAddress getCblockServerRpcAddr(
+      Configuration conf) {
+    final Optional<String> host = getHostNameFromConfigKeys(conf,
+        DFS_CBLOCK_JSCSIRPC_ADDRESS_KEY);
+
+    // If no port number is specified then we'll just try the defaultBindPort.
+    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
+        DFS_CBLOCK_JSCSIRPC_ADDRESS_KEY);
+
+    return NetUtils.createSocketAddr(
+        host.or(DFS_CBLOCK_SERVICERPC_HOSTNAME_DEFAULT) + ":" +
+            port.or(DFS_CBLOCK_JSCSI_PORT_DEFAULT));
+  }
+
+  /**
+   * Retrieve the hostname, trying the supplied config keys in order.
+   * Each config value may be absent, or if present in the format
+   * host:port (the :port part is optional).
+   *
+   * @param conf  - Conf
+   * @param keys a list of configuration key names.
+   *
+   * @return first hostname component found from the given keys, or absent.
+   * @throws IllegalArgumentException if any values are not in the 'host'
+   *             or host:port format.
+   */
+  public static Optional<String> getHostNameFromConfigKeys(Configuration conf,
+      String... keys) {
+    for (final String key : keys) {
+      final String value = conf.getTrimmed(key);
+      final Optional<String> hostName = getHostName(value);
+      if (hostName.isPresent()) {
+        return hostName;
+      }
+    }
+    return Optional.absent();
+  }
+
+  /**
+   * Gets the hostname or Indicates that it is absent.
+   * @param value host or host:port
+   * @return hostname
+   */
+  public static Optional<String> getHostName(String value) {
+    if ((value == null) || value.isEmpty()) {
+      return Optional.absent();
+    }
+    return Optional.of(HostAndPort.fromString(value).getHostText());
+  }
+
+  /**
+   * Gets the port if there is one, throws otherwise.
+   * @param value  String in host:port format.
+   * @return Port
+   */
+  public static Optional<Integer> getHostPort(String value) {
+    if((value == null) || value.isEmpty()) {
+      return Optional.absent();
+    }
+    int port = HostAndPort.fromString(value).getPortOrDefault(NO_PORT);
+    if (port == NO_PORT) {
+      return Optional.absent();
+    } else {
+      return Optional.of(port);
+    }
+  }
+
+  /**
+   * Retrieve the port number, trying the supplied config keys in order.
+   * Each config value may be absent, or if present in the format
+   * host:port (the :port part is optional).
+   *
+   * @param conf Conf
+   * @param keys a list of configuration key names.
+   *
+   * @return first port number component found from the given keys, or absent.
+   * @throws IllegalArgumentException if any values are not in the 'host'
+   *             or host:port format.
+   */
+  public static Optional<Integer> getPortNumberFromConfigKeys(
+      Configuration conf, String... keys) {
+    for (final String key : keys) {
+      final String value = conf.getTrimmed(key);
+      final Optional<Integer> hostPort = getHostPort(value);
+      if (hostPort.isPresent()) {
+        return hostPort;
+      }
+    }
+    return Optional.absent();
+  }
+
+  /**
+   * Return the list of service addresses for the Ozone SCM. This method is used
+   * by the DataNodes to determine the service instances to connect to.
+   *
+   * @param conf
+   * @return list of SCM service addresses.
+   */
+  public static Map<String, ? extends Map<String, InetSocketAddress>>
+      getScmServiceRpcAddresses(Configuration conf) {
+    final Map<String, InetSocketAddress> serviceInstances = new HashMap<>();
+    serviceInstances.put(OZONE_SCM_SERVICE_INSTANCE_ID,
+        getScmAddressForDataNodes(conf));
+
+    final Map<String, Map<String, InetSocketAddress>> services =
+        new HashMap<>();
+    services.put(OZONE_SCM_SERVICE_ID, serviceInstances);
+    return services;
+  }
+
+  /**
+   * Checks that a given value is with a range.
+   *
+   * For example, sanitizeUserArgs(17, 3, 5, 10)
+   * ensures that 17 is greater/equal than 3 * 5 and less/equal to 3 * 10.
+   *
+   * @param valueTocheck  - value to check
+   * @param baseValue     - the base value that is being used.
+   * @param minFactor     - range min - a 2 here makes us ensure that value
+   *                        valueTocheck is at least twice the baseValue.
+   * @param maxFactor     - range max
+   * @return long
+   */
+  private static long sanitizeUserArgs(long valueTocheck, long baseValue,
+      long minFactor, long maxFactor)
+      throws IllegalArgumentException {
+    if ((valueTocheck >= (baseValue * minFactor)) &&
+        (valueTocheck <= (baseValue * maxFactor))) {
+      return valueTocheck;
+    }
+    String errMsg = String.format("%d is not within min = %d or max = " +
+        "%d", valueTocheck, baseValue * minFactor, baseValue * maxFactor);
+    throw new IllegalArgumentException(errMsg);
+  }
+
+  /**
+   * Returns the interval in which the heartbeat processor thread runs.
+   *
+   * @param conf - Configuration
+   * @return long in Milliseconds.
+   */
+  public static long getScmheartbeatCheckerInterval(Configuration conf) {
+    return conf.getLong(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS,
+        ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS_DEFAULT);
+  }
+
+  /**
+   * Heartbeat Interval - Defines the heartbeat frequency from a datanode to
+   * SCM.
+   *
+   * @param conf - Ozone Config
+   * @return - HB interval in seconds.
+   */
+  public static long getScmHeartbeatInterval(Configuration conf) {
+    return conf.getTimeDuration(
+        OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS,
+        ScmConfigKeys.OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT,
+        TimeUnit.SECONDS);
+  }
+
+  /**
+   * Get the Stale Node interval, which is used by SCM to flag a datanode as
+   * stale, if the heartbeat from that node has been missing for this duration.
+   *
+   * @param conf - Configuration.
+   * @return - Long, Milliseconds to wait before flagging a node as stale.
+   */
+  public static long getStaleNodeInterval(Configuration conf) {
+
+    long staleNodeIntevalMs = conf.getLong(OZONE_SCM_STALENODE_INTERVAL_MS,
+        OZONE_SCM_STALENODE_INTERVAL_DEFAULT);
+
+    long heartbeatThreadFrequencyMs = getScmheartbeatCheckerInterval(conf);
+
+    long heartbeatIntervalMs = getScmHeartbeatInterval(conf) * 1000;
+
+
+    // Make sure that StaleNodeInterval is configured way above the frequency
+    // at which we run the heartbeat thread.
+    //
+    // Here we check that staleNodeInterval is at least five times more than the
+    // frequency at which the accounting thread is going to run.
+    try {
+      sanitizeUserArgs(staleNodeIntevalMs, heartbeatThreadFrequencyMs, 5, 1000);
+    } catch (IllegalArgumentException ex) {
+      LOG.error("Stale Node Interval MS is cannot be honored due to " +
+              "mis-configured {}. ex:  {}",
+          OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS, ex);
+      throw ex;
+    }
+
+    // Make sure that stale node value is greater than configured value that
+    // datanodes are going to send HBs.
+    try {
+      sanitizeUserArgs(staleNodeIntevalMs, heartbeatIntervalMs, 3, 1000);
+    } catch (IllegalArgumentException ex) {
+      LOG.error("Stale Node Interval MS is cannot be honored due to " +
+              "mis-configured {}. ex:  {}",
+          OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS, ex);
+      throw ex;
+    }
+    return staleNodeIntevalMs;
+  }
+
+  /**
+   * Gets the interval for dead node flagging. This has to be a value that is
+   * greater than stale node value,  and by transitive relation we also know
+   * that this value is greater than heartbeat interval and heartbeatProcess
+   * Interval.
+   *
+   * @param conf - Configuration.
+   * @return - the interval for dead node flagging.
+   */
+  public static long getDeadNodeInterval(Configuration conf) {
+    long staleNodeIntervalMs = getStaleNodeInterval(conf);
+    long deadNodeIntervalMs = conf.getLong(
+        OZONE_SCM_DEADNODE_INTERVAL_MS, OZONE_SCM_DEADNODE_INTERVAL_DEFAULT);
+
+    try {
+      // Make sure that dead nodes Ms is at least twice the time for staleNodes
+      // with a max of 1000 times the staleNodes.
+      sanitizeUserArgs(deadNodeIntervalMs, staleNodeIntervalMs, 2, 1000);
+    } catch (IllegalArgumentException ex) {
+      LOG.error("Dead Node Interval MS is cannot be honored due to " +
+              "mis-configured {}. ex:  {}",
+          OZONE_SCM_STALENODE_INTERVAL_MS, ex);
+      throw ex;
+    }
+    return deadNodeIntervalMs;
+  }
+
+  /**
+   * Returns the maximum number of heartbeat to process per loop of the process
+   * thread.
+   * @param conf Configuration
+   * @return - int -- Number of HBs to process
+   */
+  public static int getMaxHBToProcessPerLoop(Configuration conf) {
+    return conf.getInt(ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS,
+        ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT);
+  }
+
+  /**
+   * Timeout value for the RPC from Datanode to SCM, primarily used for
+   * Heartbeats and container reports.
+   *
+   * @param conf - Ozone Config
+   * @return - Rpc timeout in Milliseconds.
+   */
+  public static long getScmRpcTimeOutInMilliseconds(Configuration conf) {
+    return conf.getTimeDuration(OZONE_SCM_HEARTBEAT_RPC_TIMEOUT,
+        OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Log Warn interval.
+   *
+   * @param conf - Ozone Config
+   * @return - Log warn interval.
+   */
+  public static int getLogWarnInterval(Configuration conf) {
+    return conf.getInt(OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT,
+        OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT);
+  }
+
+  /**
+   * returns the Container port.
+   * @param conf - Conf
+   * @return port number.
+   */
+  public static int getContainerPort(Configuration conf) {
+    return conf.getInt(ScmConfigKeys.DFS_CONTAINER_IPC_PORT, ScmConfigKeys
+        .DFS_CONTAINER_IPC_PORT_DEFAULT);
+  }
+
+  /**
+   * After starting an RPC server, updates configuration with the actual
+   * listening address of that server. The listening address may be different
+   * from the configured address if, for example, the configured address uses
+   * port 0 to request use of an ephemeral port.
+   *
+   * @param conf configuration to update
+   * @param rpcAddressKey configuration key for RPC server address
+   * @param addr configured address
+   * @param rpcServer started RPC server.
+   */
+  public static InetSocketAddress updateRPCListenAddress(
+      OzoneConfiguration conf, String rpcAddressKey,
+      InetSocketAddress addr, RPC.Server rpcServer) {
+    return updateListenAddress(conf, rpcAddressKey, addr,
+        rpcServer.getListenerAddress());
+  }
+
+  /**
+   * After starting an server, updates configuration with the actual
+   * listening address of that server. The listening address may be different
+   * from the configured address if, for example, the configured address uses
+   * port 0 to request use of an ephemeral port.
+   *
+   * @param conf       configuration to update
+   * @param addressKey configuration key for RPC server address
+   * @param addr       configured address
+   * @param listenAddr the real listening address.
+   */
+  public static InetSocketAddress updateListenAddress(OzoneConfiguration conf,
+      String addressKey, InetSocketAddress addr, InetSocketAddress listenAddr) {
+    InetSocketAddress updatedAddr = new InetSocketAddress(addr.getHostString(),
+        listenAddr.getPort());
+    conf.set(addressKey,
+        addr.getHostString() + ":" + listenAddr.getPort());
+    return updatedAddr;
+  }
+
+  /**
+   * Releases a http connection if the request is not null.
+   * @param request
+   */
+  public static void releaseConnection(HttpRequestBase request) {
+    if (request != null) {
+      request.releaseConnection();
+    }
+  }
+
+  /**
+   * @return a default instance of {@link CloseableHttpClient}.
+   */
+  public static CloseableHttpClient newHttpClient() {
+    return OzoneClientUtils.newHttpClient(null);
+  }
+
+  /**
+   * Returns a {@link CloseableHttpClient} configured by given configuration.
+   * If conf is null, returns a default instance.
+   *
+   * @param conf configuration
+   * @return a {@link CloseableHttpClient} instance.
+   */
+  public static CloseableHttpClient newHttpClient(Configuration conf) {
+    int socketTimeout = OzoneConfigKeys
+        .OZONE_CLIENT_SOCKET_TIMEOUT_MS_DEFAULT;
+    int connectionTimeout = OzoneConfigKeys
+        .OZONE_CLIENT_CONNECTION_TIMEOUT_MS_DEFAULT;
+    if (conf != null) {
+      socketTimeout = conf.getInt(
+          OzoneConfigKeys.OZONE_CLIENT_SOCKET_TIMEOUT_MS,
+          OzoneConfigKeys.OZONE_CLIENT_SOCKET_TIMEOUT_MS_DEFAULT);
+      connectionTimeout = conf.getInt(
+          OzoneConfigKeys.OZONE_CLIENT_CONNECTION_TIMEOUT_MS,
+          OzoneConfigKeys.OZONE_CLIENT_CONNECTION_TIMEOUT_MS_DEFAULT);
+    }
+
+    CloseableHttpClient client = HttpClients.custom()
+        .setDefaultRequestConfig(
+            RequestConfig.custom()
+                .setSocketTimeout(socketTimeout)
+                .setConnectTimeout(connectionTimeout)
+                .build())
+        .build();
+    return client;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
new file mode 100644
index 0000000..368736a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+
+import java.util.List;
+
+/**
+ * A class that encapsulates OzoneKey.
+ */
+public class OzoneKey {
+
+  /**
+   * Name of the Volume the Key belongs to.
+   */
+  private final String volumeName;
+  /**
+   * Name of the Bucket the Key belongs to.
+   */
+  private final String bucketName;
+  /**
+   * Name of the Key.
+   */
+  private final String keyName;
+  /**
+   * Size of the data.
+   */
+  private final long dataSize;
+
+  /**
+   * All the locations of this key, in an ordered list.
+   */
+  private final List<KsmKeyLocationInfo> keyLocations;
+  /**
+   * Constructs OzoneKey from KsmKeyInfo.
+   *
+   * @param ksmKeyInfo
+   */
+  public OzoneKey(KsmKeyInfo ksmKeyInfo) {
+    this.volumeName = ksmKeyInfo.getVolumeName();
+    this.bucketName = ksmKeyInfo.getBucketName();
+    this.keyName = ksmKeyInfo.getKeyName();
+    this.dataSize = ksmKeyInfo.getDataSize();
+    this.keyLocations = ksmKeyInfo.getKeyLocationList();
+  }
+
+  /**
+   * Returns Volume Name associated with the Key.
+   *
+   * @return volumeName
+   */
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  /**
+   * Returns Bucket Name associated with the Key.
+   *
+   * @return bucketName
+   */
+  public String getBucketName(){
+    return bucketName;
+  }
+
+  /**
+   * Returns the Key Name.
+   *
+   * @return keyName
+   */
+  public String getKeyName() {
+    return keyName;
+  }
+
+  /**
+   * Returns the size of the data.
+   *
+   * @return dataSize
+   */
+  public long getDataSize() {
+    return dataSize;
+  }
+
+  /**
+   * Retruns the list of the key locations.
+   *
+   * @return key locations
+   */
+  public List<KsmKeyLocationInfo> getKeyLocations() {
+    return keyLocations;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
new file mode 100644
index 0000000..9c2ec3d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+import org.apache.hadoop.ozone.ksm.helpers.KsmOzoneAclMap;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
+
+/**
+ * A class that encapsulates OzoneVolume.
+ */
+public class OzoneVolume {
+
+  /**
+   * Admin Name of the Volume.
+   */
+  private final String adminName;
+  /**
+   * Owner of the Volume.
+   */
+  private final String ownerName;
+  /**
+   * Name of the Volume.
+   */
+  private final String volumeName;
+  /**
+   * Quota allocated for the Volume.
+   */
+  private final long quotaInBytes;
+  /**
+   * Volume ACLs.
+   */
+  private final KsmOzoneAclMap aclMap;
+
+  /**
+   * Constructs OzoneVolume from KsmVolumeArgs.
+   *
+   * @param ksmVolumeArgs
+   */
+  public OzoneVolume(KsmVolumeArgs ksmVolumeArgs) {
+    this.adminName = ksmVolumeArgs.getAdminName();
+    this.ownerName = ksmVolumeArgs.getOwnerName();
+    this.volumeName = ksmVolumeArgs.getVolume();
+    this.quotaInBytes = ksmVolumeArgs.getQuotaInBytes();
+    this.aclMap = ksmVolumeArgs.getAclMap();
+  }
+
+  /**
+   * Returns Volume's admin name.
+   *
+   * @return adminName
+   */
+  public String getAdminName() {
+    return adminName;
+  }
+
+  /**
+   * Returns Volume's owner name.
+   *
+   * @return ownerName
+   */
+  public String getOwnerName() {
+    return ownerName;
+  }
+
+  /**
+   * Returns Volume name.
+   *
+   * @return volumeName
+   */
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  /**
+   * Returns Quota allocated for the Volume in bytes.
+   *
+   * @return quotaInBytes
+   */
+  public long getQuota() {
+    return quotaInBytes;
+  }
+
+  /**
+   * Returns OzoneAcl list associated with the Volume.
+   *
+   * @return aclMap
+   */
+  public KsmOzoneAclMap getAclMap() {
+    return aclMap;
+  }
+}
\ No newline at end of file


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


[3/8] hadoop git commit: HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.

Posted by xy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
deleted file mode 100644
index 8c7ac7e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.scm.ScmConfigKeys;
-
-/**
- * This class contains constants for configuration keys used in Ozone.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public final class OzoneConfigKeys {
-  public static final String DFS_CONTAINER_IPC_PORT =
-      "dfs.container.ipc";
-  public static final int DFS_CONTAINER_IPC_PORT_DEFAULT = 50011;
-
-  /**
-   *
-   * When set to true, allocate a random free port for ozone container,
-   * so that a mini cluster is able to launch multiple containers on a node.
-   *
-   * When set to false (default), container port is fixed as specified by
-   * DFS_CONTAINER_IPC_PORT_DEFAULT.
-   */
-  public static final String DFS_CONTAINER_IPC_RANDOM_PORT =
-      "dfs.container.ipc.random.port";
-  public static final boolean DFS_CONTAINER_IPC_RANDOM_PORT_DEFAULT =
-      false;
-
-  public static final String OZONE_LOCALSTORAGE_ROOT =
-      "ozone.localstorage.root";
-  public static final String OZONE_LOCALSTORAGE_ROOT_DEFAULT = "/tmp/ozone";
-  public static final String OZONE_ENABLED =
-      "ozone.enabled";
-  public static final boolean OZONE_ENABLED_DEFAULT = false;
-  public static final String OZONE_HANDLER_TYPE_KEY =
-      "ozone.handler.type";
-  public static final String OZONE_HANDLER_TYPE_DEFAULT = "distributed";
-  public static final String OZONE_TRACE_ENABLED_KEY =
-      "ozone.trace.enabled";
-  public static final boolean OZONE_TRACE_ENABLED_DEFAULT = false;
-
-  public static final String OZONE_CONTAINER_METADATA_DIRS =
-      "ozone.container.metadata.dirs";
-
-  public static final String OZONE_METADATA_STORE_IMPL =
-      "ozone.metastore.impl";
-  public static final String OZONE_METADATA_STORE_IMPL_LEVELDB =
-      "LevelDB";
-  public static final String OZONE_METADATA_STORE_IMPL_ROCKSDB =
-      "RocksDB";
-  public static final String OZONE_METADATA_STORE_IMPL_DEFAULT =
-      OZONE_METADATA_STORE_IMPL_LEVELDB;
-
-  public static final String OZONE_KEY_CACHE = "ozone.key.cache.size";
-  public static final int OZONE_KEY_CACHE_DEFAULT = 1024;
-
-  public static final String OZONE_SCM_BLOCK_SIZE_KEY =
-      "ozone.scm.block.size";
-  public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256 * OzoneConsts.MB;
-
-  /**
-   * Ozone administrator users delimited by comma.
-   * If not set, only the user who launches an ozone service will be the
-   * admin user. This property must be set if ozone services are started by
-   * different users. Otherwise the RPC layer will reject calls from
-   * other servers which are started by users not in the list.
-   * */
-  public static final String OZONE_ADMINISTRATORS =
-      "ozone.administrators";
-
-  public static final String OZONE_CLIENT_SOCKET_TIMEOUT_MS =
-      "ozone.client.socket.timeout.ms";
-  public static final int OZONE_CLIENT_SOCKET_TIMEOUT_MS_DEFAULT = 5000;
-  public static final String OZONE_CLIENT_CONNECTION_TIMEOUT_MS =
-      "ozone.client.connection.timeout.ms";
-  public static final int OZONE_CLIENT_CONNECTION_TIMEOUT_MS_DEFAULT = 5000;
-
-  public static final String DFS_CONTAINER_RATIS_ENABLED_KEY
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
-  public static final boolean DFS_CONTAINER_RATIS_ENABLED_DEFAULT
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
-  public static final String DFS_CONTAINER_RATIS_RPC_TYPE_KEY
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY;
-  public static final String DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT;
-  /** A unique ID to identify a Ratis server. */
-  public static final String DFS_CONTAINER_RATIS_SERVER_ID =
-      "dfs.container.ratis.server.id";
-  public static final String DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR =
-      "dfs.container.ratis.datanode.storage.dir";
-
-  public static final String OZONE_SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL =
-      "ozone.web.authentication.kerberos.principal";
-  
-  /**
-   * There is no need to instantiate this class.
-   */
-  private OzoneConfigKeys() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
deleted file mode 100644
index a16e8d9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneConfiguration.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Configuration for ozone.
- */
-@InterfaceAudience.Private
-public class OzoneConfiguration extends Configuration {
-  static {
-    // adds the default resources
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-    Configuration.addDefaultResource("ozone-default.xml");
-    Configuration.addDefaultResource("ozone-site.xml");
-  }
-
-  public OzoneConfiguration() {
-  }
-
-  public OzoneConfiguration(Configuration conf) {
-    super(conf);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneKey.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneKey.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneKey.java
deleted file mode 100644
index b6b62d2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneKey.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyLocationInfo;
-
-import java.util.List;
-
-/**
- * A class that encapsulates OzoneKey.
- */
-public class OzoneKey {
-
-  /**
-   * Name of the Volume the Key belongs to.
-   */
-  private final String volumeName;
-  /**
-   * Name of the Bucket the Key belongs to.
-   */
-  private final String bucketName;
-  /**
-   * Name of the Key.
-   */
-  private final String keyName;
-  /**
-   * Size of the data.
-   */
-  private final long dataSize;
-
-  /**
-   * All the locations of this key, in an ordered list.
-   */
-  private final List<KsmKeyLocationInfo> keyLocations;
-  /**
-   * Constructs OzoneKey from KsmKeyInfo.
-   *
-   * @param ksmKeyInfo
-   */
-  public OzoneKey(KsmKeyInfo ksmKeyInfo) {
-    this.volumeName = ksmKeyInfo.getVolumeName();
-    this.bucketName = ksmKeyInfo.getBucketName();
-    this.keyName = ksmKeyInfo.getKeyName();
-    this.dataSize = ksmKeyInfo.getDataSize();
-    this.keyLocations = ksmKeyInfo.getKeyLocationList();
-  }
-
-  /**
-   * Returns Volume Name associated with the Key.
-   *
-   * @return volumeName
-   */
-  public String getVolumeName() {
-    return volumeName;
-  }
-
-  /**
-   * Returns Bucket Name associated with the Key.
-   *
-   * @return bucketName
-   */
-  public String getBucketName(){
-    return bucketName;
-  }
-
-  /**
-   * Returns the Key Name.
-   *
-   * @return keyName
-   */
-  public String getKeyName() {
-    return keyName;
-  }
-
-  /**
-   * Returns the size of the data.
-   *
-   * @return dataSize
-   */
-  public long getDataSize() {
-    return dataSize;
-  }
-
-  /**
-   * Retruns the list of the key locations.
-   *
-   * @return key locations
-   */
-  public List<KsmKeyLocationInfo> getKeyLocations() {
-    return keyLocations;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneVolume.java
deleted file mode 100644
index 3a5e886..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneVolume.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.ksm.helpers.KsmOzoneAclMap;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
-
-/**
- * A class that encapsulates OzoneVolume.
- */
-public class OzoneVolume {
-
-  /**
-   * Admin Name of the Volume.
-   */
-  private final String adminName;
-  /**
-   * Owner of the Volume.
-   */
-  private final String ownerName;
-  /**
-   * Name of the Volume.
-   */
-  private final String volumeName;
-  /**
-   * Quota allocated for the Volume.
-   */
-  private final long quotaInBytes;
-  /**
-   * Volume ACLs.
-   */
-  private final KsmOzoneAclMap aclMap;
-
-  /**
-   * Constructs OzoneVolume from KsmVolumeArgs.
-   *
-   * @param ksmVolumeArgs
-   */
-  public OzoneVolume(KsmVolumeArgs ksmVolumeArgs) {
-    this.adminName = ksmVolumeArgs.getAdminName();
-    this.ownerName = ksmVolumeArgs.getOwnerName();
-    this.volumeName = ksmVolumeArgs.getVolume();
-    this.quotaInBytes = ksmVolumeArgs.getQuotaInBytes();
-    this.aclMap = ksmVolumeArgs.getAclMap();
-  }
-
-  /**
-   * Returns Volume's admin name.
-   *
-   * @return adminName
-   */
-  public String getAdminName() {
-    return adminName;
-  }
-
-  /**
-   * Returns Volume's owner name.
-   *
-   * @return ownerName
-   */
-  public String getOwnerName() {
-    return ownerName;
-  }
-
-  /**
-   * Returns Volume name.
-   *
-   * @return volumeName
-   */
-  public String getVolumeName() {
-    return volumeName;
-  }
-
-  /**
-   * Returns Quota allocated for the Volume in bytes.
-   *
-   * @return quotaInBytes
-   */
-  public long getQuota() {
-    return quotaInBytes;
-  }
-
-  /**
-   * Returns OzoneAcl list associated with the Volume.
-   *
-   * @return aclMap
-   */
-  public KsmOzoneAclMap getAclMap() {
-    return aclMap;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index a15ce75..5e0a656 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -20,7 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.CommandDispatcher;
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ContainerReportHandler;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
index 683f3f8..872412b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
@@ -17,7 +17,7 @@
 package org.apache.hadoop.ozone.container.common.statemachine;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
 import org.apache.hadoop.ozone.protocolPB
     .StorageContainerDatanodeProtocolClientSideTranslatorPB;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
index 9023526..963b645 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.protocolPB
     .StorageContainerDatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
index 3720ec8..d35f64d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.container.common.states.datanode;
 import com.google.common.base.Strings;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/OzoneInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/OzoneInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/OzoneInputStream.java
deleted file mode 100644
index 65d5d1b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/OzoneInputStream.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone.io;
-
-import org.apache.hadoop.ozone.web.storage.ChunkGroupInputStream;
-import org.apache.hadoop.scm.storage.ChunkInputStream;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * OzoneInputStream is used to read data from Ozone.
- * It uses SCM's {@link ChunkInputStream} for reading the data.
- */
-public class OzoneInputStream extends InputStream {
-
-  private final ChunkGroupInputStream inputStream;
-
-  /**
-   * Constructs OzoneInputStream with ChunkInputStream.
-   *
-   * @param inputStream
-   */
-  public OzoneInputStream(ChunkGroupInputStream inputStream) {
-    this.inputStream = inputStream;
-  }
-
-  @Override
-  public int read() throws IOException {
-    return inputStream.read();
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    inputStream.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/OzoneOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/OzoneOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/OzoneOutputStream.java
deleted file mode 100644
index 2421c4a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/OzoneOutputStream.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone.io;
-
-import org.apache.hadoop.ozone.web.storage.ChunkGroupOutputStream;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * OzoneOutputStream is used to write data into Ozone.
- * It uses SCM's {@link ChunkGroupOutputStream} for writing the data.
- */
-public class OzoneOutputStream extends OutputStream {
-
-  private final ChunkGroupOutputStream outputStream;
-
-  /**
-   * Constructs OzoneOutputStream with ChunkGroupOutputStream.
-   *
-   * @param outputStream
-   */
-  public OzoneOutputStream(ChunkGroupOutputStream outputStream) {
-    this.outputStream = outputStream;
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    outputStream.write(b);
-  }
-
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    outputStream.write(b, off, len);
-  }
-
-  @Override
-  public synchronized void flush() throws IOException {
-    outputStream.flush();
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    //commitKey can be done here, if needed.
-    outputStream.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/package-info.java
deleted file mode 100644
index b56e367..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/io/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone.io;
-
-/**
- * This package contains Ozone I/O classes.
- */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManager.java
index b3de2e8..6c75691 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManager.java
@@ -16,8 +16,8 @@
  */
 package org.apache.hadoop.ozone.ksm;
 
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
 
 import java.io.IOException;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java
index 56a1804..585dcff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/BucketManagerImpl.java
@@ -18,8 +18,8 @@ package org.apache.hadoop.ozone.ksm;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
 import org.apache.hadoop.ozone.protocol.proto
     .KeySpaceManagerProtocolProtos.BucketInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
deleted file mode 100644
index e69300c..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone.ksm;
-
-import org.apache.hadoop.ozone.OzoneAcl;
-/**
- * KSM Constants.
- */
-public final class KSMConfigKeys {
-  /**
-   * Never constructed.
-   */
-  private KSMConfigKeys() {
-  }
-
-
-  public static final String OZONE_KSM_HANDLER_COUNT_KEY =
-      "ozone.ksm.handler.count.key";
-  public static final int OZONE_KSM_HANDLER_COUNT_DEFAULT = 20;
-
-  public static final String OZONE_KSM_ADDRESS_KEY =
-      "ozone.ksm.address";
-  public static final String OZONE_KSM_BIND_HOST_DEFAULT =
-      "0.0.0.0";
-  public static final int OZONE_KSM_PORT_DEFAULT = 9862;
-
-  public static final String OZONE_KSM_HTTP_ENABLED_KEY =
-      "ozone.ksm.http.enabled";
-  public static final String OZONE_KSM_HTTP_BIND_HOST_KEY =
-      "ozone.ksm.http-bind-host";
-  public static final String OZONE_KSM_HTTPS_BIND_HOST_KEY =
-      "ozone.ksm.https-bind-host";
-  public static final String OZONE_KSM_HTTP_ADDRESS_KEY =
-      "ozone.ksm.http-address";
-  public static final String OZONE_KSM_HTTPS_ADDRESS_KEY =
-      "ozone.ksm.https-address";
-  public static final String OZONE_KSM_KEYTAB_FILE =
-      "ozone.ksm.keytab.file";
-  public static final String OZONE_KSM_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
-  public static final int OZONE_KSM_HTTP_BIND_PORT_DEFAULT = 9874;
-  public static final int OZONE_KSM_HTTPS_BIND_PORT_DEFAULT = 9875;
-
-  // LevelDB cache file uses an off-heap cache in LevelDB of 128 MB.
-  public static final String OZONE_KSM_DB_CACHE_SIZE_MB =
-      "ozone.ksm.leveldb.cache.size.mb";
-  public static final int OZONE_KSM_DB_CACHE_SIZE_DEFAULT = 128;
-
-  public static final String OZONE_KSM_USER_MAX_VOLUME =
-      "ozone.ksm.user.max.volume";
-  public static final int OZONE_KSM_USER_MAX_VOLUME_DEFAULT = 1024;
-
-  // KSM Default user/group permissions
-  public static final String OZONE_KSM_USER_RIGHTS =
-      "ozone.ksm.user.rights";
-  public static final OzoneAcl.OzoneACLRights OZONE_KSM_USER_RIGHTS_DEFAULT =
-      OzoneAcl.OzoneACLRights.READ_WRITE;
-
-  public static final String OZONE_KSM_GROUP_RIGHTS =
-      "ozone.ksm.group.rights";
-  public static final OzoneAcl.OzoneACLRights OZONE_KSM_GROUP_RIGHTS_DEFAULT =
-      OzoneAcl.OzoneACLRights.READ_WRITE;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManager.java
index 18e6b67..753a564 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManager.java
@@ -16,8 +16,8 @@
  */
 package org.apache.hadoop.ozone.ksm;
 
-import org.apache.hadoop.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
 
 import java.io.IOException;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java
index 48e1049..f259d85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java
@@ -17,9 +17,9 @@
 package org.apache.hadoop.ozone.ksm;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyLocationInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException.ResultCodes;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
index 8b586b0..66bed52 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
@@ -24,17 +24,17 @@ import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.jmx.ServiceRuntimeInfo;
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
-import org.apache.hadoop.ksm.protocol.KeySpaceManagerProtocol;
-import org.apache.hadoop.ksm.protocolPB.KeySpaceManagerProtocolPB;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol;
+import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.protocol.proto
     .KeySpaceManagerProtocolProtos.OzoneAclInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManager.java
index d1a7903..a08898f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManager.java
@@ -17,9 +17,9 @@
 package org.apache.hadoop.ozone.ksm;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
 import org.apache.hadoop.utils.BatchOperation;
 import org.apache.hadoop.utils.MetadataStore;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManagerImpl.java
index 8bcd1f2..6be4675 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManagerImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/MetadataManagerImpl.java
@@ -21,9 +21,9 @@ import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManager.java
index 39fb08d..6ac78d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManager.java
@@ -16,7 +16,7 @@
  */
 package org.apache.hadoop.ozone.ksm;
 
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
 import org.apache.hadoop.ozone.protocol.proto
     .KeySpaceManagerProtocolProtos.OzoneAclInfo;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManagerImpl.java
index f4590a2..1ebf509 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManagerImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/VolumeManagerImpl.java
@@ -17,7 +17,7 @@
 package org.apache.hadoop.ozone.ksm;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
 import org.apache.hadoop.ozone.protocol.proto
@@ -34,10 +34,8 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
-import static org.apache.hadoop.ozone.ksm
-    .KSMConfigKeys.OZONE_KSM_USER_MAX_VOLUME_DEFAULT;
-import static org.apache.hadoop.ozone.ksm
-    .KSMConfigKeys.OZONE_KSM_USER_MAX_VOLUME;
+import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_USER_MAX_VOLUME_DEFAULT;
+import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_USER_MAX_VOLUME;
 import static org.apache.hadoop.ozone.ksm.exceptions
     .KSMException.ResultCodes;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java
index 0b131ee..46af396 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java
@@ -19,13 +19,13 @@ package org.apache.hadoop.ozone.protocolPB;
 import com.google.common.collect.Lists;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
-import org.apache.hadoop.ksm.protocol.KeySpaceManagerProtocol;
-import org.apache.hadoop.ksm.protocolPB.KeySpaceManagerProtocolPB;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol;
+import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB;
 import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
 import org.apache.hadoop.ozone.protocol.proto
     .KeySpaceManagerProtocolProtos.CreateBucketRequest;
@@ -89,8 +89,8 @@ import java.util.List;
 
 /**
  * This class is the server-side translator that forwards requests received on
- * {@link org.apache.hadoop.ksm.protocolPB.KeySpaceManagerProtocolPB} to the
- * KeySpaceManagerService server implementation.
+ * {@link org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB}
+ * to the KeySpaceManagerService server implementation.
  */
 public class KeySpaceManagerProtocolServerSideTranslatorPB implements
     KeySpaceManagerProtocolPB {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
index 9d3df77..d22ab23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.metrics2.util.MBeans;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/node/SCMNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/node/SCMNodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/node/SCMNodeManager.java
index dc7db2d..bcd3a02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/node/SCMNodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/node/SCMNodeManager.java
@@ -25,7 +25,7 @@ import org.apache.commons.collections.map.HashedMap;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.metrics2.util.MBeans;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
 import org.apache.hadoop.ozone.protocol.VersionResponse;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/OzoneHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/OzoneHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/OzoneHttpServer.java
index 192b8bc..8d0da04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/OzoneHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/OzoneHttpServer.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
index b973095..e92a0d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
@@ -21,10 +21,10 @@ package org.apache.hadoop.ozone.web.client;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.KeyInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java
index 02a00b5..df8b782 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java
@@ -23,9 +23,9 @@ import com.google.common.base.Strings;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.response.ListVolumes;
 import org.apache.hadoop.ozone.web.response.VolumeInfo;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
index f7cc649..0881e39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
@@ -21,9 +21,9 @@ package org.apache.hadoop.ozone.web.client;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.ListBuckets;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketHandler.java
index c7d3eed..e052f38 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketHandler.java
@@ -22,7 +22,7 @@ package org.apache.hadoop.ozone.web.handlers;
 
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.interfaces.Bucket;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.OzoneConsts;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java
index 5a96a84..db4a23c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.web.handlers;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.interfaces.UserAuth;
 import org.apache.hadoop.ozone.web.response.BucketInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
index 66aeb62..06fd2b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
@@ -19,10 +19,10 @@
 package org.apache.hadoop.ozone.web.handlers;
 
 import org.apache.commons.codec.binary.Hex;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.interfaces.Keys;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.response.KeyInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java
index 015fcf9..8a28c3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java
@@ -22,7 +22,7 @@ import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.interfaces.UserAuth;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java
index c9f20c2..76aa286 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.web.handlers;
 
 import com.sun.jersey.spi.container.ContainerRequest;
 import com.sun.jersey.spi.container.ContainerRequestFilter;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 
 import javax.ws.rs.core.UriBuilder;
 import javax.ws.rs.ext.Provider;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java
index 48d024c..394a2a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.web.handlers;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.interfaces.UserAuth;
 import org.apache.hadoop.ozone.web.interfaces.Volume;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
deleted file mode 100644
index 5c4263f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.web.headers;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * OZONE specific HTTP headers.
- */
-@InterfaceAudience.Private
-public final class Header {
-  public static final String OZONE_QUOTA_BYTES = "BYTES";
-  public static final String OZONE_QUOTA_MB = "MB";
-  public static final String OZONE_QUOTA_GB = "GB";
-  public static final String OZONE_QUOTA_TB = "TB";
-  public static final String OZONE_QUOTA_REMOVE = "remove";
-  public static final String OZONE_QUOTA_UNDEFINED = "undefined";
-  public static final String OZONE_EMPTY_STRING="";
-  public static final String OZONE_DEFAULT_LIST_SIZE = "1000";
-
-  public static final String OZONE_USER = "x-ozone-user";
-  public static final String OZONE_SIMPLE_AUTHENTICATION_SCHEME = "OZONE";
-  public static final String OZONE_VERSION_HEADER = "x-ozone-version";
-  public static final String OZONE_V1_VERSION_HEADER ="v1";
-
-  public static final String OZONE_LIST_QUERY_SERVICE = "service";
-  public static final String OZONE_LIST_QUERY_VOLUME = "volume";
-  public static final String OZONE_LIST_QUERY_BUCKET = "bucket";
-  public static final String OZONE_LIST_QUERY_KEY = "key";
-
-  public static final String OZONE_REQUEST_ID = "x-ozone-request-id";
-  public static final String OZONE_SERVER_NAME = "x-ozone-server-name";
-
-  public static final String OZONE_STORAGE_TYPE = "x-ozone-storage-type";
-
-  public static final String OZONE_BUCKET_VERSIONING =
-      "x-ozone-bucket-versioning";
-
-  public static final String OZONE_ACLS = "x-ozone-acls";
-  public static final String OZONE_ACL_ADD = "ADD";
-  public static final String OZONE_ACL_REMOVE = "REMOVE";
-
-  public static final String OZONE_LIST_QUERY_TAG ="info";
-  public static final String OZONE_QUOTA_QUERY_TAG ="quota";
-  public static final String CONTENT_MD5 = "Content-MD5";
-  public static final String OZONE_LIST_QUERY_PREFIX="prefix";
-  public static final String OZONE_LIST_QUERY_MAXKEYS="max-keys";
-  public static final String OZONE_LIST_QUERY_PREVKEY="prev-key";
-  public static final String OZONE_LIST_QUERY_ROOTSCAN="root-scan";
-
-  private Header() {
-    // Never constructed.
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/package-info.java
deleted file mode 100644
index abef63c..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/package-info.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/**
- * Ozone HTTP header definitions.
- */
-@InterfaceAudience.Private
-package org.apache.hadoop.ozone.web.headers;
-
-import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Bucket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Bucket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Bucket.java
index 13866f0..a123d69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Bucket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Bucket.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.ozone.web.interfaces;
 
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java
index e64c8ba..1e830b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Keys.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.ozone.web.interfaces;
 
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
index cbefba3..0cfef40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.ozone.web.interfaces;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.KeyArgs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
index e74ded7..91ede36 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/interfaces/Volume.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.web.interfaces;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
index 7b2bc53..68f115f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.web.localstorage;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.KeyArgs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
index f28555a..c0f9630 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
@@ -20,9 +20,9 @@ package org.apache.hadoop.ozone.web.localstorage;
 import com.google.common.base.Preconditions;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/LengthInputStreamMessageBodyWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/LengthInputStreamMessageBodyWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/LengthInputStreamMessageBodyWriter.java
index 502ccd5..6db49d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/LengthInputStreamMessageBodyWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/messages/LengthInputStreamMessageBodyWriter.java
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.ozone.web.messages;
 
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
 
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.MultivaluedMap;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
index c51fbd3..62cd0f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
@@ -19,10 +19,8 @@
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneBucket;
 import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
 import org.apache.hadoop.ozone.web.client.OzoneKey;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
index f9ed45f..9619ebd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.web.request;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/ChunkGroupInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/ChunkGroupInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/ChunkGroupInputStream.java
deleted file mode 100644
index 12df012..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/ChunkGroupInputStream.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.web.storage;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyLocationInfo;
-import org.apache.hadoop.scm.XceiverClientManager;
-import org.apache.hadoop.scm.XceiverClientSpi;
-import org.apache.hadoop.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.scm.storage.ChunkInputStream;
-import org.apache.hadoop.scm.storage.ContainerProtocolCalls;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Maintaining a list of ChunkInputStream. Read based on offset.
- */
-public class ChunkGroupInputStream extends InputStream {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ChunkGroupInputStream.class);
-
-  private static final int EOF = -1;
-
-  private final ArrayList<ChunkInputStreamEntry> streamEntries;
-  private int currentStreamIndex;
-
-  public ChunkGroupInputStream() {
-    streamEntries = new ArrayList<>();
-    currentStreamIndex = 0;
-  }
-
-  @VisibleForTesting
-  public synchronized int getCurrentStreamIndex() {
-    return currentStreamIndex;
-  }
-
-  @VisibleForTesting
-  public long getRemainingOfIndex(int index) {
-    return streamEntries.get(index).getRemaining();
-  }
-
-  /**
-   * Append another stream to the end of the list.
-   *
-   * @param stream the stream instance.
-   * @param length the max number of bytes that should be written to this
-   *               stream.
-   */
-  public synchronized void addStream(InputStream stream, long length) {
-    streamEntries.add(new ChunkInputStreamEntry(stream, length));
-  }
-
-
-  @Override
-  public synchronized int read() throws IOException {
-    if (streamEntries.size() <= currentStreamIndex) {
-      throw new IndexOutOfBoundsException();
-    }
-    ChunkInputStreamEntry entry = streamEntries.get(currentStreamIndex);
-    int data = entry.read();
-    return data;
-  }
-
-  @Override
-  public synchronized int read(byte[] b, int off, int len) throws IOException {
-    if (b == null) {
-      throw new NullPointerException();
-    }
-    if (off < 0 || len < 0 || len > b.length - off) {
-      throw new IndexOutOfBoundsException();
-    }
-    if (len == 0) {
-      return 0;
-    }
-    int totalReadLen = 0;
-    while (len > 0) {
-      if (streamEntries.size() <= currentStreamIndex) {
-        return totalReadLen == 0 ? EOF : totalReadLen;
-      }
-      ChunkInputStreamEntry current = streamEntries.get(currentStreamIndex);
-      int readLen = Math.min(len, (int)current.getRemaining());
-      int actualLen = current.read(b, off, readLen);
-      // this means the underlying stream has nothing at all, return
-      if (actualLen == EOF) {
-        return totalReadLen > 0? totalReadLen : EOF;
-      }
-      totalReadLen += actualLen;
-      // this means there is no more data to read beyond this point, return
-      if (actualLen != readLen) {
-        return totalReadLen;
-      }
-      off += readLen;
-      len -= readLen;
-      if (current.getRemaining() <= 0) {
-        currentStreamIndex += 1;
-      }
-    }
-    return totalReadLen;
-  }
-
-  private static class ChunkInputStreamEntry extends InputStream {
-
-    private final InputStream inputStream;
-    private final long length;
-    private long currentPosition;
-
-
-    ChunkInputStreamEntry(InputStream chunkInputStream, long length) {
-      this.inputStream = chunkInputStream;
-      this.length = length;
-      this.currentPosition = 0;
-    }
-
-    synchronized long getRemaining() {
-      return length - currentPosition;
-    }
-
-    @Override
-    public synchronized int read(byte[] b, int off, int len)
-        throws IOException {
-      int readLen = inputStream.read(b, off, len);
-      currentPosition += readLen;
-      return readLen;
-    }
-
-    @Override
-    public synchronized int read() throws IOException {
-      int data = inputStream.read();
-      currentPosition += 1;
-      return data;
-    }
-
-    @Override
-    public synchronized void close() throws IOException {
-      inputStream.close();
-    }
-  }
-
-  public static LengthInputStream getFromKsmKeyInfo(KsmKeyInfo keyInfo,
-      XceiverClientManager xceiverClientManager,
-      StorageContainerLocationProtocolClientSideTranslatorPB
-          storageContainerLocationClient, String requestId)
-      throws IOException {
-    int index = 0;
-    long length = 0;
-    String containerKey;
-    ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream();
-    for (KsmKeyLocationInfo ksmKeyLocationInfo : keyInfo.getKeyLocationList()) {
-      // check index as sanity check
-      Preconditions.checkArgument(index++ == ksmKeyLocationInfo.getIndex());
-      String containerName = ksmKeyLocationInfo.getContainerName();
-      Pipeline pipeline =
-          storageContainerLocationClient.getContainer(containerName);
-      XceiverClientSpi xceiverClient =
-          xceiverClientManager.acquireClient(pipeline);
-      boolean success = false;
-      containerKey = ksmKeyLocationInfo.getBlockID();
-      try {
-        LOG.debug("get key accessing {} {}",
-            xceiverClient.getPipeline().getContainerName(), containerKey);
-        ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation
-            .containerKeyDataForRead(
-                xceiverClient.getPipeline().getContainerName(), containerKey);
-        ContainerProtos.GetKeyResponseProto response = ContainerProtocolCalls
-            .getKey(xceiverClient, containerKeyData, requestId);
-        List<ContainerProtos.ChunkInfo> chunks =
-            response.getKeyData().getChunksList();
-        for (ContainerProtos.ChunkInfo chunk : chunks) {
-          length += chunk.getLen();
-        }
-        success = true;
-        ChunkInputStream inputStream = new ChunkInputStream(
-            containerKey, xceiverClientManager, xceiverClient,
-            chunks, requestId);
-        groupInputStream.addStream(inputStream,
-            ksmKeyLocationInfo.getLength());
-      } finally {
-        if (!success) {
-          xceiverClientManager.releaseClient(xceiverClient);
-        }
-      }
-    }
-    return new LengthInputStream(groupInputStream, length);
-  }
-}


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


[4/8] hadoop git commit: HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.

Posted by xy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestArchive.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestArchive.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestArchive.java
new file mode 100644
index 0000000..6fff47e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestArchive.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.scm;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.zip.Adler32;
+import java.util.zip.Checksum;
+
+/**
+ * Test archive creation and unpacking.
+ */
+public class TestArchive {
+  private static final int DIR_COUNT = 10;
+  private static final int SUB_DIR_COUNT = 3;
+  private static final int FILE_COUNT = 10;
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Rule
+  public TemporaryFolder outputFolder = new TemporaryFolder();
+
+  Checksum crc = new Adler32();
+
+  @Before
+  public void setUp() throws Exception {
+    Random r = new Random();
+    final int megaByte = 1024 * 1024;
+
+    for (int x = 0; x < DIR_COUNT; x++) {
+      File subdir = folder.newFolder(String.format("dir%d", x));
+      for (int y = 0; y < SUB_DIR_COUNT; y++) {
+        File targetDir = new File(subdir.getPath().concat(File.separator)
+            .concat(String.format("subdir%d%d", x, y)));
+        if(!targetDir.mkdirs()) {
+          throw new IOException("Failed to create subdirectory. " +
+              targetDir.toString());
+        }
+        for (int z = 0; z < FILE_COUNT; z++) {
+          Path temp = Paths.get(targetDir.getPath().concat(File.separator)
+              .concat(String.format("File%d.txt", z)));
+          byte[] buf = RandomStringUtils.randomAlphanumeric(r.nextInt(megaByte))
+              .getBytes("UTF-8");
+          Files.write(temp, buf);
+          crc.update(buf, 0, buf.length);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testArchive() throws Exception {
+    Checksum readCrc = new Adler32();
+    File archiveFile = new File(outputFolder.getRoot() + File.separator
+        + "test.container.zip");
+    long zipCheckSum = FileUtil.zip(folder.getRoot(), archiveFile);
+    Assert.assertTrue(zipCheckSum > 0);
+    File decomp = new File(outputFolder.getRoot() + File.separator +
+        "decompress");
+    if (!decomp.exists() && !decomp.mkdirs()) {
+      throw new IOException("Unable to create the destination directory. " +
+          decomp.getPath());
+    }
+
+    FileUtil.unZip(archiveFile, decomp);
+    String[] patterns = {"txt"};
+    Iterator<File> iter = FileUtils.iterateFiles(decomp, patterns, true);
+    int count = 0;
+    while (iter.hasNext()) {
+      count++;
+      byte[] buf = Files.readAllBytes(iter.next().toPath());
+      readCrc.update(buf, 0, buf.length);
+    }
+    Assert.assertEquals(DIR_COUNT * SUB_DIR_COUNT * FILE_COUNT, count);
+    Assert.assertEquals(crc.getValue(), readCrc.getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/package-info.java
new file mode 100644
index 0000000..9c480d6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.scm;
+/**
+ Test cases for SCM client classes.
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockConfigKeys.java
deleted file mode 100644
index 35b8961..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockConfigKeys.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.cblock;
-
-import static java.lang.Thread.NORM_PRIORITY;
-
-/**
- * This class contains constants for configuration keys used in CBlock.
- */
-public final class CBlockConfigKeys {
-  public static final String DFS_CBLOCK_SERVICERPC_ADDRESS_KEY =
-      "dfs.cblock.servicerpc-address";
-  public static final int DFS_CBLOCK_SERVICERPC_PORT_DEFAULT =
-      9810;
-  public static final String DFS_CBLOCK_SERVICERPC_HOSTNAME_DEFAULT =
-      "0.0.0.0";
-
-  public static final String DFS_CBLOCK_JSCSIRPC_ADDRESS_KEY =
-      "dfs.cblock.jscsi-address";
-
-  //The port on CBlockManager node for jSCSI to ask
-  public static final String DFS_CBLOCK_JSCSI_PORT_KEY =
-      "dfs.cblock.jscsi.port";
-  public static final int DFS_CBLOCK_JSCSI_PORT_DEFAULT =
-      9811;
-
-  public static final String DFS_CBLOCK_SERVICERPC_BIND_HOST_KEY =
-      "dfs.cblock.service.rpc-bind-host";
-  public static final String DFS_CBLOCK_JSCSIRPC_BIND_HOST_KEY =
-      "dfs.cblock.jscsi.rpc-bind-host";
-
-  // default block size is 4KB
-  public static final int DFS_CBLOCK_SERVICE_BLOCK_SIZE_DEFAULT =
-      4096;
-
-  public static final String DFS_CBLOCK_SERVICERPC_HANDLER_COUNT_KEY =
-      "dfs.storage.service.handler.count";
-  public static final int DFS_CBLOCK_SERVICERPC_HANDLER_COUNT_DEFAULT = 10;
-
-  public static final String DFS_CBLOCK_SERVICE_LEVELDB_PATH_KEY =
-      "dfs.cblock.service.leveldb.path";
-  //TODO : find a better place
-  public static final String DFS_CBLOCK_SERVICE_LEVELDB_PATH_DEFAULT =
-      "/tmp/cblock_levelDB.dat";
-
-
-  public static final String DFS_CBLOCK_DISK_CACHE_PATH_KEY =
-      "dfs.cblock.disk.cache.path";
-  public static final String DFS_CBLOCK_DISK_CACHE_PATH_DEFAULT =
-      "/tmp/cblockCacheDB";
-  /**
-   * Setting this flag to true makes the block layer compute a sha256 hash of
-   * the data and log that information along with block ID. This is very
-   * useful for doing trace based simulation of various workloads. Since it is
-   * computing a hash for each block this could be expensive, hence default
-   * is false.
-   */
-  public static final String DFS_CBLOCK_TRACE_IO = "dfs.cblock.trace.io";
-  public static final boolean DFS_CBLOCK_TRACE_IO_DEFAULT = false;
-
-  public static final String DFS_CBLOCK_ENABLE_SHORT_CIRCUIT_IO =
-      "dfs.cblock.short.circuit.io";
-  public static final boolean DFS_CBLOCK_ENABLE_SHORT_CIRCUIT_IO_DEFAULT =
-      false;
-
-  /**
-   * Cache size in 1000s of entries. 256 indicates 256 * 1024.
-   */
-  public static final String DFS_CBLOCK_CACHE_QUEUE_SIZE_KB =
-      "dfs.cblock.cache.cache.size.in.kb";
-  public static final int DFS_CBLOCK_CACHE_QUEUE_SIZE_KB_DEFAULT = 256;
-
-  /**
-   *  Minimum Number of threads that cache pool will use for background I/O.
-   */
-  public static final String DFS_CBLOCK_CACHE_CORE_MIN_POOL_SIZE =
-      "dfs.cblock.cache.core.min.pool.size";
-  public static final int DFS_CBLOCK_CACHE_CORE_MIN_POOL_SIZE_DEFAULT = 16;
-
-  /**
-   *  Maximum Number of threads that cache pool will use for background I/O.
-   */
-
-  public static final String DFS_CBLOCK_CACHE_MAX_POOL_SIZE =
-      "dfs.cblock.cache.max.pool.size";
-  public static final int DFS_CBLOCK_CACHE_MAX_POOL_SIZE_DEFAULT = 256;
-
-  /**
-   * Number of seconds to keep the Thread alive when it is idle.
-   */
-  public static final String DFS_CBLOCK_CACHE_KEEP_ALIVE_SECONDS =
-      "dfs.cblock.cache.keep.alive.seconds";
-  public static final long DFS_CBLOCK_CACHE_KEEP_ALIVE_SECONDS_DEFAULT = 60;
-
-  /**
-   * Priority of cache flusher thread, affecting the relative performance of
-   * write and read.
-   */
-  public static final String DFS_CBLOCK_CACHE_THREAD_PRIORITY =
-      "dfs.cblock.cache.thread.priority";
-  public static final int DFS_CBLOCK_CACHE_THREAD_PRIORITY_DEFAULT =
-      NORM_PRIORITY;
-
-  /**
-   * Block Buffer size in terms of blockID entries, 512 means 512 blockIDs.
-   */
-  public static final String DFS_CBLOCK_CACHE_BLOCK_BUFFER_SIZE =
-      "dfs.cblock.cache.block.buffer.size";
-  public static final int DFS_CBLOCK_CACHE_BLOCK_BUFFER_SIZE_DEFAULT = 512;
-
-  public static final String DFS_CBLOCK_BLOCK_BUFFER_FLUSH_INTERVAL_SECONDS =
-      "dfs.cblock.block.buffer.flush.interval.seconds";
-  public static final int
-      DFS_CBLOCK_BLOCK_BUFFER_FLUSH_INTERVAL_SECONDS_DEFAULT = 60;
-
-  // jscsi server settings
-  public static final String DFS_CBLOCK_JSCSI_SERVER_ADDRESS_KEY =
-      "dfs.cblock.jscsi.server.address";
-  public static final String DFS_CBLOCK_JSCSI_SERVER_ADDRESS_DEFAULT =
-      "127.0.0.1";
-  public static final String DFS_CBLOCK_JSCSI_CBLOCK_SERVER_ADDRESS_KEY =
-      "dfs.cblock.jscsi.cblock.server.address";
-  public static final String DFS_CBLOCK_JSCSI_CBLOCK_SERVER_ADDRESS_DEFAULT =
-      "127.0.0.1";
-
-  // to what address cblock server should talk to scm?
-  public static final String DFS_CBLOCK_SCM_IPADDRESS_KEY =
-      "dfs.cblock.scm.ipaddress";
-  public static final String DFS_CBLOCK_SCM_IPADDRESS_DEFAULT =
-      "127.0.0.1";
-  public static final String DFS_CBLOCK_SCM_PORT_KEY =
-      "dfs.cblock.scm.port";
-  public static final int DFS_CBLOCK_SCM_PORT_DEFAULT = 9860;
-
-  public static final String DFS_CBLOCK_CONTAINER_SIZE_GB_KEY =
-      "dfs.cblock.container.size";
-  public static final int DFS_CBLOCK_CONTAINER_SIZE_GB_DEFAULT =
-      5;
-
-  // LevelDB cache file uses an off-heap cache in LevelDB of 256 MB.
-  public static final String DFS_CBLOCK_CACHE_LEVELDB_CACHE_SIZE_MB_KEY =
-      "dfs.cblock.cache.leveldb.cache.size.mb";
-  public static final int DFS_CBLOCK_CACHE_LEVELDB_CACHE_SIZE_MB_DEFAULT = 256;
-
-  /**
-   * Cache does an best case attempt to write a block to a container.
-   * At some point of time, we will need to handle the case where we did try
-   * 64K times and is till not able to write to the container.
-   *
-   * TODO: We will need cBlock Server to allow us to do a remapping of the
-   * block location in case of failures, at that point we should reduce the
-   * retry count to a more normal number. This is approximately 18 hours of
-   * retry.
-   */
-  public static final String DFS_CBLOCK_CACHE_MAX_RETRY_KEY =
-      "dfs.cblock.cache.max.retry";
-  public static final int DFS_CBLOCK_CACHE_MAX_RETRY_DEFAULT =
-      64 * 1024;
-
-  private CBlockConfigKeys() {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockManager.java
index 144c3bb..d7349cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockManager.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.cblock.protocolPB.CBlockServiceProtocolPB;
 import org.apache.hadoop.cblock.protocolPB
     .CBlockServiceProtocolServerSideTranslatorPB;
 import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.scm.XceiverClientManager;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/client/CBlockVolumeClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/client/CBlockVolumeClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/client/CBlockVolumeClient.java
index f70e8a4..90a16ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/client/CBlockVolumeClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/client/CBlockVolumeClient.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.cblock.protocolPB.CBlockServiceProtocolPB;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.security.UserGroupInformation;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
index 5b76179..e6dff98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
@@ -31,12 +31,13 @@ import com.sun.jersey.api.container.ContainerFactory;
 import com.sun.jersey.api.core.ApplicationAdapter;
 
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ksm.protocolPB
+import org.apache.hadoop.ozone.ksm.protocolPB
     .KeySpaceManagerProtocolClientSideTranslatorPB;
-import org.apache.hadoop.ksm.protocolPB.KeySpaceManagerProtocolPB;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.scm.protocolPB.ScmBlockLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.scm.protocolPB
+    .ScmBlockLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.scm.protocolPB.ScmBlockLocationProtocolPB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,7 +48,8 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.OzoneConfiguration;
-import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.scm.protocolPB
+    .StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.ozone.web.handlers.ServiceFilter;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java
index bfdecce..5114298 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java
@@ -25,7 +25,7 @@ import io.netty.channel.SimpleChannelInboundHandler;
 import io.netty.handler.codec.http.HttpRequest;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.netty.ObjectStoreJerseyContainer;
 import org.apache.hadoop.ozone.web.netty.RequestDispatchObjectStoreChannelHandler;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
index 725b20f..d2aaf73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
-import org.apache.hadoop.ozone.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneBucket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneBucket.java
deleted file mode 100644
index 51eefdc..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneBucket.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ozone.OzoneConsts.Versioning;
-
-import java.util.List;
-
-/**
- * A class that encapsulates OzoneBucket.
- */
-public class OzoneBucket {
-
-  /**
-   * Name of the volume in which the bucket belongs to.
-   */
-  private final String volumeName;
-  /**
-   * Name of the bucket.
-   */
-  private final String bucketName;
-  /**
-   * Bucket ACLs.
-   */
-  private final List<OzoneAcl> acls;
-
-  /**
-   * Type of storage to be used for this bucket.
-   * [RAM_DISK, SSD, DISK, ARCHIVE]
-   */
-  private final StorageType storageType;
-
-  /**
-   * Bucket Version flag.
-   */
-  private final Versioning versioning;
-
-
-  /**
-   * Constructs OzoneBucket from KsmBucketInfo.
-   *
-   * @param ksmBucketInfo
-   */
-  public OzoneBucket(KsmBucketInfo ksmBucketInfo) {
-    this.volumeName = ksmBucketInfo.getVolumeName();
-    this.bucketName = ksmBucketInfo.getBucketName();
-    this.acls = ksmBucketInfo.getAcls();
-    this.storageType = ksmBucketInfo.getStorageType();
-    this.versioning = ksmBucketInfo.getIsVersionEnabled() ?
-        Versioning.ENABLED : Versioning.DISABLED;
-  }
-
-  /**
-   * Returns Volume Name.
-   *
-   * @return volumeName
-   */
-  public String getVolumeName() {
-    return volumeName;
-  }
-
-  /**
-   * Returns Bucket Name.
-   *
-   * @return bucketName
-   */
-  public String getBucketName() {
-    return bucketName;
-  }
-
-  /**
-   * Returns ACL's associated with the Bucket.
-   *
-   * @return acls
-   */
-  public List<OzoneAcl> getAcls() {
-    return acls;
-  }
-
-  /**
-   * Returns StorageType of the Bucket.
-   *
-   * @return storageType
-   */
-  public StorageType getStorageType() {
-    return storageType;
-  }
-
-  /**
-   * Returns Versioning associated with the Bucket.
-   *
-   * @return versioning
-   */
-  public Versioning getVersioning() {
-    return versioning;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClient.java
deleted file mode 100644
index dd52a57..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClient.java
+++ /dev/null
@@ -1,414 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ozone.OzoneConsts.Versioning;
-import org.apache.hadoop.ozone.io.OzoneInputStream;
-import org.apache.hadoop.ozone.io.OzoneOutputStream;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * OzoneClient can connect to a Ozone Cluster and
- * perform basic operations.
- */
-public interface OzoneClient {
-
-  /**
-   * Creates a new Volume.
-   *
-   * @param volumeName Name of the Volume
-   *
-   * @throws IOException
-   */
-  void createVolume(String volumeName)
-      throws IOException;
-
-  /**
-   * Creates a new Volume, with owner set.
-   *
-   * @param volumeName Name of the Volume
-   * @param owner Owner to be set for Volume
-   *
-   * @throws IOException
-   */
-  void createVolume(String volumeName, String owner)
-      throws IOException;
-
-  /**
-   * Creates a new Volume, with owner and quota set.
-   *
-   * @param volumeName Name of the Volume
-   * @param owner Owner to be set for Volume
-   * @param acls ACLs to be added to the Volume
-   *
-   * @throws IOException
-   */
-  void createVolume(String volumeName, String owner,
-                    OzoneAcl... acls)
-      throws IOException;
-
-  /**
-   * Creates a new Volume, with owner and quota set.
-   *
-   * @param volumeName Name of the Volume
-   * @param owner Owner to be set for Volume
-   * @param quota Volume Quota
-   *
-   * @throws IOException
-   */
-  void createVolume(String volumeName, String owner,
-                    long quota)
-      throws IOException;
-
-  /**
-   * Creates a new Volume, with owner and quota set.
-   *
-   * @param volumeName Name of the Volume
-   * @param owner Owner to be set for Volume
-   * @param quota Volume Quota
-   * @param acls ACLs to be added to the Volume
-   *
-   * @throws IOException
-   */
-  void createVolume(String volumeName, String owner,
-                    long quota, OzoneAcl... acls)
-      throws IOException;
-
-  /**
-   * Sets the owner of the volume.
-   *
-   * @param volumeName Name of the Volume
-   * @param owner to be set for the Volume
-   *
-   * @throws IOException
-   */
-  void setVolumeOwner(String volumeName, String owner) throws IOException;
-
-  /**
-   * Set Volume Quota.
-   *
-   * @param volumeName Name of the Volume
-   * @param quota Quota to be set for the Volume
-   *
-   * @throws IOException
-   */
-  void setVolumeQuota(String volumeName, long quota)
-      throws IOException;
-
-  /**
-   * Returns {@link OzoneVolume}.
-   *
-   * @param volumeName Name of the Volume
-   *
-   * @return KsmVolumeArgs
-   *
-   * @throws OzoneVolume
-   * */
-  OzoneVolume getVolumeDetails(String volumeName)
-      throws IOException;
-
-  /**
-   * Checks if a Volume exists and the user with a role specified has access
-   * to the Volume.
-   *
-   * @param volumeName Name of the Volume
-   * @param acl requested acls which needs to be checked for access
-   *
-   * @return Boolean - True if the user with a role can access the volume.
-   * This is possible for owners of the volume and admin users
-   *
-   * @throws IOException
-   */
-  boolean checkVolumeAccess(String volumeName, OzoneAcl acl)
-      throws IOException;
-
-  /**
-   * Deletes an Empty Volume.
-   *
-   * @param volumeName Name of the Volume
-   *
-   * @throws IOException
-   */
-  void deleteVolume(String volumeName) throws IOException;
-
-  /**
-   * Returns the List of Volumes owned by current user.
-   *
-   * @param volumePrefix Volume prefix to match
-   *
-   * @return KsmVolumeArgs Iterator
-   *
-   * @throws IOException
-   */
-  Iterator<OzoneVolume> listVolumes(String volumePrefix)
-      throws IOException;
-
-  /**
-   * Returns the List of Volumes owned by the specific user.
-   *
-   * @param volumePrefix Volume prefix to match
-   * @param user User Name
-   *
-   * @return KsmVolumeArgs Iterator
-   *
-   * @throws IOException
-   */
-  Iterator<OzoneVolume> listVolumes(String volumePrefix, String user)
-      throws IOException;
-
-  /**
-   * Creates a new Bucket in the Volume.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   *
-   * @throws IOException
-   */
-  void createBucket(String volumeName, String bucketName)
-      throws IOException;
-
-  /**
-   * Creates a new Bucket in the Volume, with versioning set.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   * @param versioning Bucket versioning
-   *
-   * @throws IOException
-   */
-  void createBucket(String volumeName, String bucketName,
-                    Versioning versioning)
-      throws IOException;
-
-  /**
-   * Creates a new Bucket in the Volume, with storage type set.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   * @param storageType StorageType for the Bucket
-   *
-   * @throws IOException
-   */
-  void createBucket(String volumeName, String bucketName,
-                    StorageType storageType)
-      throws IOException;
-
-  /**
-   * Creates a new Bucket in the Volume, with ACLs set.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   * @param acls OzoneAcls for the Bucket
-   *
-   * @throws IOException
-   */
-  void createBucket(String volumeName, String bucketName,
-                           OzoneAcl... acls)
-      throws IOException;
-
-
-  /**
-   * Creates a new Bucket in the Volume, with versioning
-   * storage type and ACLs set.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   * @param storageType StorageType for the Bucket
-   *
-   * @throws IOException
-   */
-  void createBucket(String volumeName, String bucketName,
-                           OzoneConsts.Versioning versioning,
-                           StorageType storageType, OzoneAcl... acls)
-      throws IOException;
-
-  /**
-   * Adds or Removes ACLs from a Bucket.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   *
-   * @throws IOException
-   */
-  void addBucketAcls(String volumeName, String bucketName,
-                     List<OzoneAcl> addAcls)
-      throws IOException;
-
-  /**
-   * Adds or Removes ACLs from a Bucket.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   *
-   * @throws IOException
-   */
-  void removeBucketAcls(String volumeName, String bucketName,
-                        List<OzoneAcl> removeAcls)
-      throws IOException;
-
-
-  /**
-   * Enables or disables Bucket Versioning.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   *
-   * @throws IOException
-   */
-  void setBucketVersioning(String volumeName, String bucketName,
-                           Versioning versioning)
-      throws IOException;
-
-  /**
-   * Sets the Storage Class of a Bucket.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   *
-   * @throws IOException
-   */
-  void setBucketStorageType(String volumeName, String bucketName,
-                            StorageType storageType)
-      throws IOException;
-
-  /**
-   * Deletes a bucket if it is empty.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   *
-   * @throws IOException
-   */
-  void deleteBucket(String volumeName, String bucketName)
-      throws IOException;
-
-  /**
-   * true if the bucket exists and user has read access
-   * to the bucket else throws Exception.
-   *
-   * @param volumeName Name of the Volume
-   *
-   * @throws IOException
-   */
-  void checkBucketAccess(String volumeName, String bucketName)
-      throws IOException;
-
-    /**
-     * Returns {@link OzoneBucket}.
-     *
-     * @param volumeName Name of the Volume
-     * @param bucketName Name of the Bucket
-     *
-     * @return OzoneBucket
-     *
-     * @throws IOException
-     */
-  OzoneBucket getBucketDetails(String volumeName, String bucketName)
-        throws IOException;
-
-  /**
-   * Returns the List of Buckets in the Volume.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketPrefix Bucket prefix to match
-   *
-   * @return KsmVolumeArgs Iterator
-   *
-   * @throws IOException
-   */
-  Iterator<OzoneBucket> listBuckets(String volumeName, String bucketPrefix)
-      throws IOException;
-
-  /**
-   * Writes a key in an existing bucket.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   * @param size Size of the data
-   *
-   * @return OutputStream
-   *
-   */
-  OzoneOutputStream createKey(String volumeName, String bucketName,
-                              String keyName, long size)
-      throws IOException;
-
-  /**
-   * Reads a key from an existing bucket.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   *
-   * @return LengthInputStream
-   *
-   * @throws IOException
-   */
-  OzoneInputStream getKey(String volumeName, String bucketName, String keyName)
-      throws IOException;
-
-
-  /**
-   * Deletes an existing key.
-   *
-   * @param volumeName Name of the Volume
-   *
-   * @throws IOException
-   */
-  void deleteKey(String volumeName, String bucketName, String keyName)
-      throws IOException;
-
-
-  /**
-   * Returns list of {@link OzoneKey} in Volume/Bucket.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   *
-   * @return OzoneKey
-   *
-   * @throws IOException
-   */
-  List<OzoneKey> listKeys(String volumeName, String bucketName,
-                            String keyPrefix)
-      throws IOException;
-
-
-  /**
-   * Get OzoneKey.
-   *
-   * @param volumeName Name of the Volume
-   * @param bucketName Name of the Bucket
-   * @param keyName Key name
-   *
-   * @return OzoneKey
-   *
-   * @throws IOException
-   */
-  OzoneKey getKeyDetails(String volumeName, String bucketName,
-                        String keyName)
-      throws IOException;
-
-  /**
-   * Close and release the resources.
-   */
-  void close() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientFactory.java
deleted file mode 100644
index 7866f58..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientFactory.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.conf.Configuration;
-
-import java.io.IOException;
-
-/**
- * Factory class to create different types of OzoneClients.
- */
-public final class OzoneClientFactory {
-
-  /**
-   * Private constructor, class is not meant to be initialized.
-   */
-  private OzoneClientFactory(){}
-
-  private static Configuration configuration;
-
-  /**
-   * Returns an OzoneClient which will use RPC protocol to perform
-   * client operations.
-   *
-   * @return OzoneClient
-   * @throws IOException
-   */
-  public static OzoneClient getRpcClient() throws IOException {
-    return new OzoneClientImpl(getConfiguration());
-  }
-
-  /**
-   * Sets the configuration, which will be used while creating OzoneClient.
-   *
-   * @param conf
-   */
-  public static void setConfiguration(Configuration conf) {
-    configuration = conf;
-  }
-
-  /**
-   * Returns the configuration if it's already set, else creates a new
-   * {@link OzoneConfiguration} and returns it.
-   *
-   * @return Configuration
-   */
-  private static synchronized Configuration getConfiguration() {
-    if(configuration == null) {
-      setConfiguration(new OzoneConfiguration());
-    }
-    return configuration;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientImpl.java
deleted file mode 100644
index feb4586..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientImpl.java
+++ /dev/null
@@ -1,570 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
-import org.apache.hadoop.ksm.protocolPB
-    .KeySpaceManagerProtocolClientSideTranslatorPB;
-import org.apache.hadoop.ksm.protocolPB
-    .KeySpaceManagerProtocolPB;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.web.storage.ChunkGroupInputStream;
-import org.apache.hadoop.ozone.io.OzoneInputStream;
-import org.apache.hadoop.ozone.io.OzoneOutputStream;
-import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts.Versioning;
-import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
-import org.apache.hadoop.ozone.web.storage.ChunkGroupOutputStream;
-import org.apache.hadoop.scm.ScmConfigKeys;
-import org.apache.hadoop.scm.XceiverClientManager;
-import org.apache.hadoop.scm.protocolPB
-    .StorageContainerLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.scm.protocolPB
-    .StorageContainerLocationProtocolPB;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Collectors;
-
-/**
- * Ozone Client Implementation, it connects to KSM, SCM and DataNode
- * to execute client calls. This uses RPC protocol for communication
- * with the servers.
- */
-public class OzoneClientImpl implements OzoneClient, Closeable {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(OzoneClient.class);
-
-  private final StorageContainerLocationProtocolClientSideTranslatorPB
-      storageContainerLocationClient;
-  private final KeySpaceManagerProtocolClientSideTranslatorPB
-      keySpaceManagerClient;
-  private final XceiverClientManager xceiverClientManager;
-  private final int chunkSize;
-
-
-  private final UserGroupInformation ugi;
-  private final OzoneAcl.OzoneACLRights userRights;
-  private final OzoneAcl.OzoneACLRights groupRights;
-
-  /**
-   * Creates OzoneClientImpl instance with new OzoneConfiguration.
-   *
-   * @throws IOException
-   */
-  public OzoneClientImpl() throws IOException {
-    this(new OzoneConfiguration());
-  }
-
-   /**
-    * Creates OzoneClientImpl instance with the given configuration.
-    *
-    * @param conf
-    *
-    * @throws IOException
-    */
-  public OzoneClientImpl(Configuration conf) throws IOException {
-    Preconditions.checkNotNull(conf);
-    this.ugi = UserGroupInformation.getCurrentUser();
-    this.userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS,
-        KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT);
-    this.groupRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS,
-        KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS_DEFAULT);
-
-    long scmVersion =
-        RPC.getProtocolVersion(StorageContainerLocationProtocolPB.class);
-    InetSocketAddress scmAddress =
-        OzoneClientUtils.getScmAddressForClients(conf);
-    RPC.setProtocolEngine(conf, StorageContainerLocationProtocolPB.class,
-        ProtobufRpcEngine.class);
-    this.storageContainerLocationClient =
-        new StorageContainerLocationProtocolClientSideTranslatorPB(
-            RPC.getProxy(StorageContainerLocationProtocolPB.class, scmVersion,
-                scmAddress, UserGroupInformation.getCurrentUser(), conf,
-                NetUtils.getDefaultSocketFactory(conf),
-                Client.getRpcTimeout(conf)));
-
-    long ksmVersion =
-        RPC.getProtocolVersion(KeySpaceManagerProtocolPB.class);
-    InetSocketAddress ksmAddress = OzoneClientUtils.getKsmAddress(conf);
-    RPC.setProtocolEngine(conf, KeySpaceManagerProtocolPB.class,
-        ProtobufRpcEngine.class);
-    this.keySpaceManagerClient =
-        new KeySpaceManagerProtocolClientSideTranslatorPB(
-            RPC.getProxy(KeySpaceManagerProtocolPB.class, ksmVersion,
-                ksmAddress, UserGroupInformation.getCurrentUser(), conf,
-                NetUtils.getDefaultSocketFactory(conf),
-                Client.getRpcTimeout(conf)));
-
-    this.xceiverClientManager = new XceiverClientManager(conf);
-
-    int configuredChunkSize = conf.getInt(
-        ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
-        ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT);
-    if(configuredChunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) {
-      LOG.warn("The chunk size ({}) is not allowed to be more than"
-              + " the maximum size ({}),"
-              + " resetting to the maximum size.",
-          configuredChunkSize, ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE);
-      chunkSize = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE;
-    } else {
-      chunkSize = configuredChunkSize;
-    }
-  }
-
-  @Override
-  public void createVolume(String volumeName)
-      throws IOException {
-    createVolume(volumeName, ugi.getUserName());
-  }
-
-  @Override
-  public void createVolume(String volumeName, String owner)
-      throws IOException {
-
-    createVolume(volumeName, owner, OzoneConsts.MAX_QUOTA_IN_BYTES,
-        (OzoneAcl[])null);
-  }
-
-  @Override
-  public void createVolume(String volumeName, String owner,
-                           OzoneAcl... acls)
-      throws IOException {
-    createVolume(volumeName, owner, OzoneConsts.MAX_QUOTA_IN_BYTES, acls);
-  }
-
-  @Override
-  public void createVolume(String volumeName, String owner,
-                           long quota)
-      throws IOException {
-    createVolume(volumeName, owner, quota, (OzoneAcl[])null);
-  }
-
-  @Override
-  public void createVolume(String volumeName, String owner,
-                           long quota, OzoneAcl... acls)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(owner);
-    Preconditions.checkNotNull(quota);
-    Preconditions.checkState(quota >= 0);
-    OzoneAcl userAcl =
-        new OzoneAcl(OzoneAcl.OzoneACLType.USER,
-            owner, userRights);
-    KsmVolumeArgs.Builder builder = KsmVolumeArgs.newBuilder();
-    builder.setAdminName(ugi.getUserName())
-        .setOwnerName(owner)
-        .setVolume(volumeName)
-        .setQuotaInBytes(quota)
-        .addOzoneAcls(KSMPBHelper.convertOzoneAcl(userAcl));
-
-    List<OzoneAcl> listOfAcls = new ArrayList<>();
-
-    //Group ACLs of the User
-    List<String> userGroups = Arrays.asList(UserGroupInformation
-        .createRemoteUser(owner).getGroupNames());
-    userGroups.stream().forEach((group) -> listOfAcls.add(
-        new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights)));
-
-    //ACLs passed as argument
-    if(acls != null) {
-      listOfAcls.addAll(Arrays.asList(acls));
-    }
-
-    //Remove duplicates and set
-    for (OzoneAcl ozoneAcl :
-        listOfAcls.stream().distinct().collect(Collectors.toList())) {
-      builder.addOzoneAcls(KSMPBHelper.convertOzoneAcl(ozoneAcl));
-    }
-
-    LOG.info("Creating Volume: {}, with {} as owner and quota set to {} bytes.",
-        volumeName, owner, quota);
-    keySpaceManagerClient.createVolume(builder.build());
-  }
-
-  @Override
-  public void setVolumeOwner(String volumeName, String owner)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(owner);
-    keySpaceManagerClient.setOwner(volumeName, owner);
-  }
-
-  @Override
-  public void setVolumeQuota(String volumeName, long quota)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(quota);
-    Preconditions.checkState(quota >= 0);
-    keySpaceManagerClient.setQuota(volumeName, quota);
-  }
-
-  @Override
-  public OzoneVolume getVolumeDetails(String volumeName)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    KsmVolumeArgs volumeArgs =
-        keySpaceManagerClient.getVolumeInfo(volumeName);
-    return new OzoneVolume(volumeArgs);
-  }
-
-  @Override
-  public boolean checkVolumeAccess(String volumeName, OzoneAcl acl)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    return keySpaceManagerClient.checkVolumeAccess(volumeName,
-        KSMPBHelper.convertOzoneAcl(acl));
-  }
-
-  @Override
-  public void deleteVolume(String volumeName)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    keySpaceManagerClient.deleteVolume(volumeName);
-  }
-
-  @Override
-  public Iterator<OzoneVolume> listVolumes(String volumePrefix)
-      throws IOException {
-    throw new UnsupportedOperationException("Not yet implemented.");
-  }
-
-  @Override
-  public Iterator<OzoneVolume> listVolumes(String volumePrefix,
-                                             String user)
-      throws IOException {
-    throw new UnsupportedOperationException("Not yet implemented.");
-  }
-
-  @Override
-  public void createBucket(String volumeName, String bucketName)
-      throws IOException {
-    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
-        StorageType.DEFAULT, (OzoneAcl[])null);
-  }
-
-  @Override
-  public void createBucket(String volumeName, String bucketName,
-                           Versioning versioning)
-      throws IOException {
-    createBucket(volumeName, bucketName, versioning,
-        StorageType.DEFAULT, (OzoneAcl[])null);
-  }
-
-  @Override
-  public void createBucket(String volumeName, String bucketName,
-                           StorageType storageType)
-      throws IOException {
-    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
-        storageType, (OzoneAcl[])null);
-  }
-
-  @Override
-  public void createBucket(String volumeName, String bucketName,
-                           OzoneAcl... acls)
-      throws IOException {
-    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
-        StorageType.DEFAULT, acls);
-  }
-
-  @Override
-  public void createBucket(String volumeName, String bucketName,
-                           Versioning versioning, StorageType storageType,
-                           OzoneAcl... acls)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(versioning);
-    Preconditions.checkNotNull(storageType);
-
-    KsmBucketInfo.Builder builder = KsmBucketInfo.newBuilder();
-    builder.setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setStorageType(storageType)
-        .setIsVersionEnabled(getBucketVersioningProtobuf(
-        versioning));
-
-    String owner = ugi.getUserName();
-    final List<OzoneAcl> listOfAcls = new ArrayList<>();
-
-    //User ACL
-    OzoneAcl userAcl =
-        new OzoneAcl(OzoneAcl.OzoneACLType.USER,
-            owner, userRights);
-    listOfAcls.add(userAcl);
-
-    //Group ACLs of the User
-    List<String> userGroups = Arrays.asList(UserGroupInformation
-        .createRemoteUser(owner).getGroupNames());
-    userGroups.stream().forEach((group) -> listOfAcls.add(
-        new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights)));
-
-    //ACLs passed as argument
-    if(acls != null) {
-      Arrays.stream(acls).forEach((acl) -> listOfAcls.add(acl));
-    }
-
-    //Remove duplicates and set
-    builder.setAcls(listOfAcls.stream().distinct()
-        .collect(Collectors.toList()));
-    LOG.info("Creating Bucket: {}/{}, with Versioning {} and " +
-        "Storage Type set to {}", volumeName, bucketName, versioning,
-        storageType);
-    keySpaceManagerClient.createBucket(builder.build());
-  }
-
-  /**
-   * Converts OzoneConts.Versioning enum to boolean.
-   *
-   * @param version
-   * @return corresponding boolean value
-   */
-  private boolean getBucketVersioningProtobuf(
-      Versioning version) {
-    if(version != null) {
-      switch(version) {
-      case ENABLED:
-        return true;
-      case NOT_DEFINED:
-      case DISABLED:
-      default:
-        return false;
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public void addBucketAcls(String volumeName, String bucketName,
-                            List<OzoneAcl> addAcls)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(addAcls);
-    KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
-    builder.setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setAddAcls(addAcls);
-    keySpaceManagerClient.setBucketProperty(builder.build());
-  }
-
-  @Override
-  public void removeBucketAcls(String volumeName, String bucketName,
-                               List<OzoneAcl> removeAcls)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(removeAcls);
-    KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
-    builder.setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setRemoveAcls(removeAcls);
-    keySpaceManagerClient.setBucketProperty(builder.build());
-  }
-
-  @Override
-  public void setBucketVersioning(String volumeName, String bucketName,
-                                  Versioning versioning)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(versioning);
-    KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
-    builder.setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setIsVersionEnabled(getBucketVersioningFlag(
-            versioning));
-    keySpaceManagerClient.setBucketProperty(builder.build());
-  }
-
-  @Override
-  public void setBucketStorageType(String volumeName, String bucketName,
-                                   StorageType storageType)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(storageType);
-    KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
-    builder.setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setStorageType(storageType);
-    keySpaceManagerClient.setBucketProperty(builder.build());
-  }
-
-  @Override
-  public void deleteBucket(String volumeName, String bucketName)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    keySpaceManagerClient.deleteBucket(volumeName, bucketName);
-  }
-
-  @Override
-  public void checkBucketAccess(String volumeName, String bucketName)
-      throws IOException {
-    throw new UnsupportedOperationException("Not yet implemented.");
-  }
-
-  @Override
-  public OzoneBucket getBucketDetails(String volumeName,
-                                        String bucketName)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    KsmBucketInfo bucketInfo =
-        keySpaceManagerClient.getBucketInfo(volumeName, bucketName);
-    return new OzoneBucket(bucketInfo);
-  }
-
-  @Override
-  public Iterator<OzoneBucket> listBuckets(String volumeName,
-                                            String bucketPrefix)
-      throws IOException {
-    throw new UnsupportedOperationException("Not yet implemented.");
-  }
-
-  @Override
-  public OzoneOutputStream createKey(String volumeName, String bucketName,
-                                     String keyName, long size)
-      throws IOException {
-    String requestId = UUID.randomUUID().toString();
-    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
-        .setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setKeyName(keyName)
-        .setDataSize(size)
-        .build();
-
-    KsmKeyInfo keyInfo = keySpaceManagerClient.allocateKey(keyArgs);
-    ChunkGroupOutputStream  groupOutputStream =
-        ChunkGroupOutputStream.getFromKsmKeyInfo(keyInfo, xceiverClientManager,
-        storageContainerLocationClient, chunkSize, requestId);
-    return new OzoneOutputStream(groupOutputStream);
-  }
-
-  @Override
-  public OzoneInputStream getKey(String volumeName, String bucketName,
-                                 String keyName)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(keyName);
-    String requestId = UUID.randomUUID().toString();
-    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
-        .setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setKeyName(keyName)
-        .build();
-    KsmKeyInfo keyInfo = keySpaceManagerClient.lookupKey(keyArgs);
-    LengthInputStream lengthInputStream =
-        ChunkGroupInputStream.getFromKsmKeyInfo(
-        keyInfo, xceiverClientManager, storageContainerLocationClient,
-        requestId);
-    return new OzoneInputStream(
-        (ChunkGroupInputStream)lengthInputStream.getWrappedStream());
-  }
-
-  @Override
-  public void deleteKey(String volumeName, String bucketName,
-                        String keyName)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(keyName);
-    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
-        .setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setKeyName(keyName)
-        .build();
-    keySpaceManagerClient.deleteKey(keyArgs);
-  }
-
-  @Override
-  public List<OzoneKey> listKeys(String volumeName, String bucketName,
-                                   String keyPrefix)
-      throws IOException {
-    throw new UnsupportedOperationException("Not yet implemented.");
-  }
-
-  @Override
-  public OzoneKey getKeyDetails(String volumeName, String bucketName,
-                                  String keyName)
-      throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(keyName);
-    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
-        .setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setKeyName(keyName)
-        .build();
-    KsmKeyInfo keyInfo =
-        keySpaceManagerClient.lookupKey(keyArgs);
-    return new OzoneKey(keyInfo);
-  }
-
-  /**
-   * Converts Versioning to boolean.
-   *
-   * @param version
-   * @return corresponding boolean value
-   */
-  private boolean getBucketVersioningFlag(
-      Versioning version) {
-    if(version != null) {
-      switch(version) {
-      case ENABLED:
-        return true;
-      case DISABLED:
-      case NOT_DEFINED:
-      default:
-        return false;
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public void close() throws IOException {
-    IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
-    IOUtils.cleanupWithLogger(LOG, keySpaceManagerClient);
-    IOUtils.cleanupWithLogger(LOG, xceiverClientManager);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientUtils.java
deleted file mode 100644
index d92f49f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/OzoneClientUtils.java
+++ /dev/null
@@ -1,705 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import com.google.common.base.Optional;
-
-import com.google.common.net.HostAndPort;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.scm.ScmConfigKeys;
-import org.apache.http.client.config.RequestConfig;
-import org.apache.http.client.methods.HttpRequestBase;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.cblock.CBlockConfigKeys
-    .DFS_CBLOCK_SERVICERPC_ADDRESS_KEY;
-import static org.apache.hadoop.cblock.CBlockConfigKeys
-    .DFS_CBLOCK_SERVICERPC_HOSTNAME_DEFAULT;
-import static org.apache.hadoop.cblock.CBlockConfigKeys
-    .DFS_CBLOCK_SERVICERPC_PORT_DEFAULT;
-import static org.apache.hadoop.cblock.CBlockConfigKeys
-    .DFS_CBLOCK_JSCSIRPC_ADDRESS_KEY;
-import static org.apache.hadoop.cblock.CBlockConfigKeys
-    .DFS_CBLOCK_JSCSI_PORT_DEFAULT;
-
-import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY;
-import static org.apache.hadoop.ozone.ksm.KSMConfigKeys
-    .OZONE_KSM_BIND_HOST_DEFAULT;
-import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_PORT_DEFAULT;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_DEADNODE_INTERVAL_DEFAULT;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_DEADNODE_INTERVAL_MS;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS;
-
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS;
-
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_STALENODE_INTERVAL_DEFAULT;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_STALENODE_INTERVAL_MS;
-
-/**
- * Utility methods for Ozone and Container Clients.
- *
- * The methods to retrieve SCM service endpoints assume there is a single
- * SCM service instance. This will change when we switch to replicated service
- * instances for redundancy.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public final class OzoneClientUtils {
-  private static final Logger LOG = LoggerFactory.getLogger(
-      OzoneClientUtils.class);
-  private static final int NO_PORT = -1;
-
-  /**
-   * The service ID of the solitary Ozone SCM service.
-   */
-  public static final String OZONE_SCM_SERVICE_ID = "OzoneScmService";
-  public static final String OZONE_SCM_SERVICE_INSTANCE_ID =
-      "OzoneScmServiceInstance";
-
-  private OzoneClientUtils() {
-    // Never constructed
-  }
-
-  /**
-   * Retrieve the socket addresses of all storage container managers.
-   *
-   * @param conf
-   * @return A collection of SCM addresses
-   * @throws IllegalArgumentException If the configuration is invalid
-   */
-  public static Collection<InetSocketAddress> getSCMAddresses(
-      Configuration conf) throws IllegalArgumentException {
-    Collection<InetSocketAddress> addresses =
-        new HashSet<InetSocketAddress>();
-    Collection<String> names =
-        conf.getTrimmedStringCollection(ScmConfigKeys.OZONE_SCM_NAMES);
-    if (names == null || names.isEmpty()) {
-      throw new IllegalArgumentException(ScmConfigKeys.OZONE_SCM_NAMES
-          + " need to be a set of valid DNS names or IP addresses."
-          + " Null or empty address list found.");
-    }
-
-    final com.google.common.base.Optional<Integer>
-        defaultPort =  com.google.common.base.Optional.of(ScmConfigKeys
-        .OZONE_SCM_DEFAULT_PORT);
-    for (String address : names) {
-      com.google.common.base.Optional<String> hostname =
-          OzoneClientUtils.getHostName(address);
-      if (!hostname.isPresent()) {
-        throw new IllegalArgumentException("Invalid hostname for SCM: "
-            + hostname);
-      }
-      com.google.common.base.Optional<Integer> port =
-          OzoneClientUtils.getHostPort(address);
-      InetSocketAddress addr = NetUtils.createSocketAddr(hostname.get(),
-          port.or(defaultPort.get()));
-      addresses.add(addr);
-    }
-    return addresses;
-  }
-
-  /**
-   * Retrieve the socket address that should be used by clients to connect
-   * to the SCM.
-   *
-   * @param conf
-   * @return Target InetSocketAddress for the SCM client endpoint.
-   */
-  public static InetSocketAddress getScmAddressForClients(Configuration conf) {
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
-
-    if (!host.isPresent()) {
-      throw new IllegalArgumentException(
-          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY +
-          " must be defined. See" +
-          " https://wiki.apache.org/hadoop/Ozone#Configuration for details" +
-          " on configuring Ozone.");
-    }
-
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(host.get() + ":" +
-        port.or(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the socket address that should be used by clients to connect
-   * to the SCM for block service. If
-   * {@link ScmConfigKeys#OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY} is not defined
-   * then {@link ScmConfigKeys#OZONE_SCM_CLIENT_ADDRESS_KEY} is used.
-   *
-   * @param conf
-   * @return Target InetSocketAddress for the SCM block client endpoint.
-   * @throws IllegalArgumentException if configuration is not defined.
-   */
-  public static InetSocketAddress getScmAddressForBlockClients(
-      Configuration conf) {
-    Optional<String> host = getHostNameFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY);
-
-    if (!host.isPresent()) {
-      host = getHostNameFromConfigKeys(conf,
-              ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
-      if (!host.isPresent()) {
-        throw new IllegalArgumentException(
-                ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY +
-                        " must be defined. See" +
-                        " https://wiki.apache.org/hadoop/Ozone#Configuration for details" +
-                        " on configuring Ozone.");
-      }
-    }
-
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(host.get() + ":" +
-        port.or(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the socket address that should be used by DataNodes to connect
-   * to the SCM.
-   *
-   * @param conf
-   * @return Target InetSocketAddress for the SCM service endpoint.
-   */
-  public static InetSocketAddress getScmAddressForDataNodes(
-      Configuration conf) {
-    // We try the following settings in decreasing priority to retrieve the
-    // target host.
-    // - OZONE_SCM_DATANODE_ADDRESS_KEY
-    // - OZONE_SCM_CLIENT_ADDRESS_KEY
-    //
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY,
-        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
-
-    if (!host.isPresent()) {
-      throw new IllegalArgumentException(
-          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY +
-          " must be defined. See" +
-          " https://wiki.apache.org/hadoop/Ozone#Configuration for details" +
-          " on configuring Ozone.");
-    }
-
-    // If no port number is specified then we'll just try the defaultBindPort.
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY);
-
-    InetSocketAddress addr = NetUtils.createSocketAddr(host.get() + ":" +
-        port.or(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
-
-    return addr;
-  }
-
-  /**
-   * Retrieve the socket address that should be used by clients to connect
-   * to the SCM.
-   *
-   * @param conf
-   * @return Target InetSocketAddress for the SCM client endpoint.
-   */
-  public static InetSocketAddress getScmClientBindAddress(
-      Configuration conf) {
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY);
-
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(
-        host.or(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_DEFAULT) + ":" +
-            port.or(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the socket address that should be used by clients to connect
-   * to the SCM Block service.
-   *
-   * @param conf
-   * @return Target InetSocketAddress for the SCM block client endpoint.
-   */
-  public static InetSocketAddress getScmBlockClientBindAddress(
-      Configuration conf) {
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_BIND_HOST_KEY);
-
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(
-        host.or(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_BIND_HOST_DEFAULT) +
-            ":" + port.or(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the socket address that should be used by DataNodes to connect
-   * to the SCM.
-   *
-   * @param conf
-   * @return Target InetSocketAddress for the SCM service endpoint.
-   */
-  public static InetSocketAddress getScmDataNodeBindAddress(
-      Configuration conf) {
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY);
-
-    // If no port number is specified then we'll just try the defaultBindPort.
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(
-        host.or(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_DEFAULT) + ":" +
-            port.or(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
-  }
-
-
-  /**
-   * Retrieve the socket address that is used by KSM.
-   * @param conf
-   * @return Target InetSocketAddress for the SCM service endpoint.
-   */
-  public static InetSocketAddress getKsmAddress(
-      Configuration conf) {
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        OZONE_KSM_ADDRESS_KEY);
-
-    // If no port number is specified then we'll just try the defaultBindPort.
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        OZONE_KSM_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(
-        host.or(OZONE_KSM_BIND_HOST_DEFAULT) + ":" +
-            port.or(OZONE_KSM_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the socket address that is used by CBlock Service.
-   * @param conf
-   * @return Target InetSocketAddress for the CBlock Service endpoint.
-   */
-  public static InetSocketAddress getCblockServiceRpcAddr(
-      Configuration conf) {
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        DFS_CBLOCK_SERVICERPC_ADDRESS_KEY);
-
-    // If no port number is specified then we'll just try the defaultBindPort.
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        DFS_CBLOCK_SERVICERPC_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(
-        host.or(DFS_CBLOCK_SERVICERPC_HOSTNAME_DEFAULT) + ":" +
-            port.or(DFS_CBLOCK_SERVICERPC_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the socket address that is used by CBlock Server.
-   * @param conf
-   * @return Target InetSocketAddress for the CBlock Server endpoint.
-   */
-  public static InetSocketAddress getCblockServerRpcAddr(
-      Configuration conf) {
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        DFS_CBLOCK_JSCSIRPC_ADDRESS_KEY);
-
-    // If no port number is specified then we'll just try the defaultBindPort.
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        DFS_CBLOCK_JSCSIRPC_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(
-        host.or(DFS_CBLOCK_SERVICERPC_HOSTNAME_DEFAULT) + ":" +
-            port.or(DFS_CBLOCK_JSCSI_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the hostname, trying the supplied config keys in order.
-   * Each config value may be absent, or if present in the format
-   * host:port (the :port part is optional).
-   *
-   * @param conf  - Conf
-   * @param keys a list of configuration key names.
-   *
-   * @return first hostname component found from the given keys, or absent.
-   * @throws IllegalArgumentException if any values are not in the 'host'
-   *             or host:port format.
-   */
-  public static Optional<String> getHostNameFromConfigKeys(Configuration conf,
-      String... keys) {
-    for (final String key : keys) {
-      final String value = conf.getTrimmed(key);
-      final Optional<String> hostName = getHostName(value);
-      if (hostName.isPresent()) {
-        return hostName;
-      }
-    }
-    return Optional.absent();
-  }
-
-  /**
-   * Gets the hostname or Indicates that it is absent.
-   * @param value host or host:port
-   * @return hostname
-   */
-  public static Optional<String> getHostName(String value) {
-    if ((value == null) || value.isEmpty()) {
-      return Optional.absent();
-    }
-    return Optional.of(HostAndPort.fromString(value).getHostText());
-  }
-
-  /**
-   * Gets the port if there is one, throws otherwise.
-   * @param value  String in host:port format.
-   * @return Port
-   */
-  public static Optional<Integer> getHostPort(String value) {
-    if((value == null) || value.isEmpty()) {
-      return Optional.absent();
-    }
-    int port = HostAndPort.fromString(value).getPortOrDefault(NO_PORT);
-    if (port == NO_PORT) {
-      return Optional.absent();
-    } else {
-      return Optional.of(port);
-    }
-  }
-
-  /**
-   * Retrieve the port number, trying the supplied config keys in order.
-   * Each config value may be absent, or if present in the format
-   * host:port (the :port part is optional).
-   *
-   * @param conf Conf
-   * @param keys a list of configuration key names.
-   *
-   * @return first port number component found from the given keys, or absent.
-   * @throws IllegalArgumentException if any values are not in the 'host'
-   *             or host:port format.
-   */
-  public static Optional<Integer> getPortNumberFromConfigKeys(
-      Configuration conf, String... keys) {
-    for (final String key : keys) {
-      final String value = conf.getTrimmed(key);
-      final Optional<Integer> hostPort = getHostPort(value);
-      if (hostPort.isPresent()) {
-        return hostPort;
-      }
-    }
-    return Optional.absent();
-  }
-
-  /**
-   * Return the list of service addresses for the Ozone SCM. This method is used
-   * by the DataNodes to determine the service instances to connect to.
-   *
-   * @param conf
-   * @return list of SCM service addresses.
-   */
-  public static Map<String, ? extends Map<String, InetSocketAddress>>
-      getScmServiceRpcAddresses(Configuration conf) {
-    final Map<String, InetSocketAddress> serviceInstances = new HashMap<>();
-    serviceInstances.put(OZONE_SCM_SERVICE_INSTANCE_ID,
-        getScmAddressForDataNodes(conf));
-
-    final Map<String, Map<String, InetSocketAddress>> services =
-        new HashMap<>();
-    services.put(OZONE_SCM_SERVICE_ID, serviceInstances);
-    return services;
-  }
-
-  /**
-   * Checks that a given value is with a range.
-   *
-   * For example, sanitizeUserArgs(17, 3, 5, 10)
-   * ensures that 17 is greater/equal than 3 * 5 and less/equal to 3 * 10.
-   *
-   * @param valueTocheck  - value to check
-   * @param baseValue     - the base value that is being used.
-   * @param minFactor     - range min - a 2 here makes us ensure that value
-   *                        valueTocheck is at least twice the baseValue.
-   * @param maxFactor     - range max
-   * @return long
-   */
-  private static long sanitizeUserArgs(long valueTocheck, long baseValue,
-      long minFactor, long maxFactor)
-      throws IllegalArgumentException {
-    if ((valueTocheck >= (baseValue * minFactor)) &&
-        (valueTocheck <= (baseValue * maxFactor))) {
-      return valueTocheck;
-    }
-    String errMsg = String.format("%d is not within min = %d or max = " +
-        "%d", valueTocheck, baseValue * minFactor, baseValue * maxFactor);
-    throw new IllegalArgumentException(errMsg);
-  }
-
-  /**
-   * Returns the interval in which the heartbeat processor thread runs.
-   *
-   * @param conf - Configuration
-   * @return long in Milliseconds.
-   */
-  public static long getScmheartbeatCheckerInterval(Configuration conf) {
-    return conf.getLong(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS,
-        ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS_DEFAULT);
-  }
-
-  /**
-   * Heartbeat Interval - Defines the heartbeat frequency from a datanode to
-   * SCM.
-   *
-   * @param conf - Ozone Config
-   * @return - HB interval in seconds.
-   */
-  public static long getScmHeartbeatInterval(Configuration conf) {
-    return conf.getTimeDuration(
-        OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS,
-        ScmConfigKeys.OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT,
-        TimeUnit.SECONDS);
-  }
-
-  /**
-   * Get the Stale Node interval, which is used by SCM to flag a datanode as
-   * stale, if the heartbeat from that node has been missing for this duration.
-   *
-   * @param conf - Configuration.
-   * @return - Long, Milliseconds to wait before flagging a node as stale.
-   */
-  public static long getStaleNodeInterval(Configuration conf) {
-
-    long staleNodeIntevalMs = conf.getLong(OZONE_SCM_STALENODE_INTERVAL_MS,
-        OZONE_SCM_STALENODE_INTERVAL_DEFAULT);
-
-    long heartbeatThreadFrequencyMs = getScmheartbeatCheckerInterval(conf);
-
-    long heartbeatIntervalMs = getScmHeartbeatInterval(conf) * 1000;
-
-
-    // Make sure that StaleNodeInterval is configured way above the frequency
-    // at which we run the heartbeat thread.
-    //
-    // Here we check that staleNodeInterval is at least five times more than the
-    // frequency at which the accounting thread is going to run.
-    try {
-      sanitizeUserArgs(staleNodeIntevalMs, heartbeatThreadFrequencyMs, 5, 1000);
-    } catch (IllegalArgumentException ex) {
-      LOG.error("Stale Node Interval MS is cannot be honored due to " +
-              "mis-configured {}. ex:  {}",
-          OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS, ex);
-      throw ex;
-    }
-
-    // Make sure that stale node value is greater than configured value that
-    // datanodes are going to send HBs.
-    try {
-      sanitizeUserArgs(staleNodeIntevalMs, heartbeatIntervalMs, 3, 1000);
-    } catch (IllegalArgumentException ex) {
-      LOG.error("Stale Node Interval MS is cannot be honored due to " +
-              "mis-configured {}. ex:  {}",
-          OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS, ex);
-      throw ex;
-    }
-    return staleNodeIntevalMs;
-  }
-
-  /**
-   * Gets the interval for dead node flagging. This has to be a value that is
-   * greater than stale node value,  and by transitive relation we also know
-   * that this value is greater than heartbeat interval and heartbeatProcess
-   * Interval.
-   *
-   * @param conf - Configuration.
-   * @return - the interval for dead node flagging.
-   */
-  public static long getDeadNodeInterval(Configuration conf) {
-    long staleNodeIntervalMs = getStaleNodeInterval(conf);
-    long deadNodeIntervalMs = conf.getLong(
-        OZONE_SCM_DEADNODE_INTERVAL_MS, OZONE_SCM_DEADNODE_INTERVAL_DEFAULT);
-
-    try {
-      // Make sure that dead nodes Ms is at least twice the time for staleNodes
-      // with a max of 1000 times the staleNodes.
-      sanitizeUserArgs(deadNodeIntervalMs, staleNodeIntervalMs, 2, 1000);
-    } catch (IllegalArgumentException ex) {
-      LOG.error("Dead Node Interval MS is cannot be honored due to " +
-              "mis-configured {}. ex:  {}",
-          OZONE_SCM_STALENODE_INTERVAL_MS, ex);
-      throw ex;
-    }
-    return deadNodeIntervalMs;
-  }
-
-  /**
-   * Returns the maximum number of heartbeat to process per loop of the process
-   * thread.
-   * @param conf Configuration
-   * @return - int -- Number of HBs to process
-   */
-  public static int getMaxHBToProcessPerLoop(Configuration conf) {
-    return conf.getInt(ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS,
-        ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT);
-  }
-
-  /**
-   * Timeout value for the RPC from Datanode to SCM, primarily used for
-   * Heartbeats and container reports.
-   *
-   * @param conf - Ozone Config
-   * @return - Rpc timeout in Milliseconds.
-   */
-  public static long getScmRpcTimeOutInMilliseconds(Configuration conf) {
-    return conf.getTimeDuration(OZONE_SCM_HEARTBEAT_RPC_TIMEOUT,
-        OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
-  }
-
-  /**
-   * Log Warn interval.
-   *
-   * @param conf - Ozone Config
-   * @return - Log warn interval.
-   */
-  public static int getLogWarnInterval(Configuration conf) {
-    return conf.getInt(OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT,
-        OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT);
-  }
-
-  /**
-   * returns the Container port.
-   * @param conf - Conf
-   * @return port number.
-   */
-  public static int getContainerPort(Configuration conf) {
-    return conf.getInt(ScmConfigKeys.DFS_CONTAINER_IPC_PORT, ScmConfigKeys
-        .DFS_CONTAINER_IPC_PORT_DEFAULT);
-  }
-
-  /**
-   * After starting an RPC server, updates configuration with the actual
-   * listening address of that server. The listening address may be different
-   * from the configured address if, for example, the configured address uses
-   * port 0 to request use of an ephemeral port.
-   *
-   * @param conf configuration to update
-   * @param rpcAddressKey configuration key for RPC server address
-   * @param addr configured address
-   * @param rpcServer started RPC server.
-   */
-  public static InetSocketAddress updateRPCListenAddress(
-      OzoneConfiguration conf, String rpcAddressKey,
-      InetSocketAddress addr, RPC.Server rpcServer) {
-    return updateListenAddress(conf, rpcAddressKey, addr,
-        rpcServer.getListenerAddress());
-  }
-
-  /**
-   * After starting an server, updates configuration with the actual
-   * listening address of that server. The listening address may be different
-   * from the configured address if, for example, the configured address uses
-   * port 0 to request use of an ephemeral port.
-   *
-   * @param conf       configuration to update
-   * @param addressKey configuration key for RPC server address
-   * @param addr       configured address
-   * @param listenAddr the real listening address.
-   */
-  public static InetSocketAddress updateListenAddress(OzoneConfiguration conf,
-      String addressKey, InetSocketAddress addr, InetSocketAddress listenAddr) {
-    InetSocketAddress updatedAddr = new InetSocketAddress(addr.getHostString(),
-        listenAddr.getPort());
-    conf.set(addressKey,
-        addr.getHostString() + ":" + listenAddr.getPort());
-    return updatedAddr;
-  }
-
-  /**
-   * Releases a http connection if the request is not null.
-   * @param request
-   */
-  public static void releaseConnection(HttpRequestBase request) {
-    if (request != null) {
-      request.releaseConnection();
-    }
-  }
-
-  /**
-   * @return a default instance of {@link CloseableHttpClient}.
-   */
-  public static CloseableHttpClient newHttpClient() {
-    return OzoneClientUtils.newHttpClient(null);
-  }
-
-  /**
-   * Returns a {@link CloseableHttpClient} configured by given configuration.
-   * If conf is null, returns a default instance.
-   *
-   * @param conf configuration
-   * @return a {@link CloseableHttpClient} instance.
-   */
-  public static CloseableHttpClient newHttpClient(Configuration conf) {
-    int socketTimeout = OzoneConfigKeys
-        .OZONE_CLIENT_SOCKET_TIMEOUT_MS_DEFAULT;
-    int connectionTimeout = OzoneConfigKeys
-        .OZONE_CLIENT_CONNECTION_TIMEOUT_MS_DEFAULT;
-    if (conf != null) {
-      socketTimeout = conf.getInt(
-          OzoneConfigKeys.OZONE_CLIENT_SOCKET_TIMEOUT_MS,
-          OzoneConfigKeys.OZONE_CLIENT_SOCKET_TIMEOUT_MS_DEFAULT);
-      connectionTimeout = conf.getInt(
-          OzoneConfigKeys.OZONE_CLIENT_CONNECTION_TIMEOUT_MS,
-          OzoneConfigKeys.OZONE_CLIENT_CONNECTION_TIMEOUT_MS_DEFAULT);
-    }
-
-    CloseableHttpClient client = HttpClients.custom()
-        .setDefaultRequestConfig(
-            RequestConfig.custom()
-                .setSocketTimeout(socketTimeout)
-                .setConnectTimeout(connectionTimeout)
-                .build())
-        .build();
-    return client;
-  }
-}


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


[2/8] hadoop git commit: HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.

Posted by xy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/ChunkGroupOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/ChunkGroupOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/ChunkGroupOutputStream.java
deleted file mode 100644
index 59c2639..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/ChunkGroupOutputStream.java
+++ /dev/null
@@ -1,325 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.web.storage;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.Result;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyLocationInfo;
-import org.apache.hadoop.scm.XceiverClientManager;
-import org.apache.hadoop.scm.XceiverClientSpi;
-import org.apache.hadoop.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
-import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.scm.storage.ChunkOutputStream;
-import org.apache.hadoop.scm.storage.ContainerProtocolCalls;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * Maintaining a list of ChunkInputStream. Write based on offset.
- *
- * Note that this may write to multiple containers in one write call. In case
- * that first container succeeded but later ones failed, the succeeded writes
- * are not rolled back.
- *
- * TODO : currently not support multi-thread access.
- */
-public class ChunkGroupOutputStream extends OutputStream {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ChunkGroupOutputStream.class);
-
-  // array list's get(index) is O(1)
-  private final ArrayList<ChunkOutputStreamEntry> streamEntries;
-  private int currentStreamIndex;
-  private long totalSize;
-  private long byteOffset;
-
-  //This has to be removed once HDFS-11888 is resolved.
-  //local cache which will have list of created container names.
-  private static Set<String> containersCreated = new HashSet<>();
-
-  public ChunkGroupOutputStream() {
-    this.streamEntries = new ArrayList<>();
-    this.currentStreamIndex = 0;
-    this.totalSize = 0;
-    this.byteOffset = 0;
-  }
-
-  @VisibleForTesting
-  public long getByteOffset() {
-    return byteOffset;
-  }
-
-  /**
-   * Append another stream to the end of the list. Note that the streams are not
-   * actually created to this point, only enough meta data about the stream is
-   * stored. When something is to be actually written to the stream, the stream
-   * will be created (if not already).
-   *
-   * @param containerKey the key to store in the container
-   * @param key the ozone key
-   * @param xceiverClientManager xceiver manager instance
-   * @param xceiverClient xceiver manager instance
-   * @param requestID the request id
-   * @param chunkSize the chunk size for this key chunks
-   * @param length the total length of this key
-   */
-  public synchronized void addStream(String containerKey, String key,
-      XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
-      String requestID, int chunkSize, long length) {
-    streamEntries.add(new ChunkOutputStreamEntry(containerKey, key,
-        xceiverClientManager, xceiverClient, requestID, chunkSize, length));
-    totalSize += length;
-  }
-
-  @VisibleForTesting
-  public synchronized void addStream(OutputStream outputStream, long length) {
-    streamEntries.add(new ChunkOutputStreamEntry(outputStream, length));
-    totalSize += length;
-  }
-
-  @Override
-  public synchronized void write(int b) throws IOException {
-    if (streamEntries.size() <= currentStreamIndex) {
-      throw new IndexOutOfBoundsException();
-    }
-    ChunkOutputStreamEntry entry = streamEntries.get(currentStreamIndex);
-    entry.write(b);
-    if (entry.getRemaining() <= 0) {
-      currentStreamIndex += 1;
-    }
-    byteOffset += 1;
-  }
-
-  /**
-   * Try to write the bytes sequence b[off:off+len) to streams.
-   *
-   * NOTE: Throws exception if the data could not fit into the remaining space.
-   * In which case nothing will be written.
-   * TODO:May need to revisit this behaviour.
-   *
-   * @param b byte data
-   * @param off starting offset
-   * @param len length to write
-   * @throws IOException
-   */
-  @Override
-  public synchronized void write(byte[] b, int off, int len)
-      throws IOException {
-    if (b == null) {
-      throw new NullPointerException();
-    }
-    if ((off < 0) || (off > b.length) || (len < 0) ||
-        ((off + len) > b.length) || ((off + len) < 0)) {
-      throw new IndexOutOfBoundsException();
-    }
-    if (len == 0) {
-      return;
-    }
-    if (streamEntries.size() <= currentStreamIndex) {
-      throw new IOException("Write out of stream range! stream index:" +
-          currentStreamIndex);
-    }
-    if (totalSize - byteOffset < len) {
-      throw new IOException("Can not write " + len + " bytes with only " +
-          (totalSize - byteOffset) + " byte space");
-    }
-    while (len > 0) {
-      // in theory, this condition should never violate due the check above
-      // still do a sanity check.
-      Preconditions.checkArgument(currentStreamIndex < streamEntries.size());
-      ChunkOutputStreamEntry current = streamEntries.get(currentStreamIndex);
-      int writeLen = Math.min(len, (int)current.getRemaining());
-      current.write(b, off, writeLen);
-      if (current.getRemaining() <= 0) {
-        currentStreamIndex += 1;
-      }
-      len -= writeLen;
-      off += writeLen;
-      byteOffset += writeLen;
-    }
-  }
-
-  @Override
-  public synchronized void flush() throws IOException {
-    for (int i = 0; i <= currentStreamIndex; i++) {
-      streamEntries.get(i).flush();
-    }
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    for (ChunkOutputStreamEntry entry : streamEntries) {
-      entry.close();
-    }
-  }
-
-  private static class ChunkOutputStreamEntry extends OutputStream {
-    private OutputStream outputStream;
-    private final String containerKey;
-    private final String key;
-    private final XceiverClientManager xceiverClientManager;
-    private final XceiverClientSpi xceiverClient;
-    private final String requestId;
-    private final int chunkSize;
-    // total number of bytes that should be written to this stream
-    private final long length;
-    // the current position of this stream 0 <= currentPosition < length
-    private long currentPosition;
-
-    ChunkOutputStreamEntry(String containerKey, String key,
-        XceiverClientManager xceiverClientManager,
-        XceiverClientSpi xceiverClient, String requestId, int chunkSize,
-        long length) {
-      this.outputStream = null;
-      this.containerKey = containerKey;
-      this.key = key;
-      this.xceiverClientManager = xceiverClientManager;
-      this.xceiverClient = xceiverClient;
-      this.requestId = requestId;
-      this.chunkSize = chunkSize;
-
-      this.length = length;
-      this.currentPosition = 0;
-    }
-
-    /**
-     * For testing purpose, taking a some random created stream instance.
-     * @param  outputStream a existing writable output stream
-     * @param  length the length of data to write to the stream
-     */
-    ChunkOutputStreamEntry(OutputStream outputStream, long length) {
-      this.outputStream = outputStream;
-      this.containerKey = null;
-      this.key = null;
-      this.xceiverClientManager = null;
-      this.xceiverClient = null;
-      this.requestId = null;
-      this.chunkSize = -1;
-
-      this.length = length;
-      this.currentPosition = 0;
-    }
-
-    long getLength() {
-      return length;
-    }
-
-    long getRemaining() {
-      return length - currentPosition;
-    }
-
-    private synchronized void checkStream() {
-      if (this.outputStream == null) {
-        this.outputStream = new ChunkOutputStream(containerKey,
-            key, xceiverClientManager, xceiverClient,
-            requestId, chunkSize);
-      }
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-      checkStream();
-      outputStream.write(b);
-      this.currentPosition += 1;
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-      checkStream();
-      outputStream.write(b, off, len);
-      this.currentPosition += len;
-    }
-
-    @Override
-    public void flush() throws IOException {
-      if (this.outputStream != null) {
-        this.outputStream.flush();
-      }
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (this.outputStream != null) {
-        this.outputStream.close();
-      }
-    }
-  }
-
-  public static ChunkGroupOutputStream getFromKsmKeyInfo(
-      KsmKeyInfo keyInfo, XceiverClientManager xceiverClientManager,
-      StorageContainerLocationProtocolClientSideTranslatorPB
-          storageContainerLocationClient,
-      int chunkSize, String requestId) throws IOException {
-    // TODO: the following createContainer and key writes may fail, in which
-    // case we should revert the above allocateKey to KSM.
-    // check index as sanity check
-    int index = 0;
-    String containerKey;
-    ChunkGroupOutputStream groupOutputStream = new ChunkGroupOutputStream();
-    for (KsmKeyLocationInfo subKeyInfo : keyInfo.getKeyLocationList()) {
-      containerKey = subKeyInfo.getBlockID();
-
-      Preconditions.checkArgument(index++ == subKeyInfo.getIndex());
-      String containerName = subKeyInfo.getContainerName();
-      Pipeline pipeline =
-          storageContainerLocationClient.getContainer(containerName);
-      XceiverClientSpi xceiverClient =
-          xceiverClientManager.acquireClient(pipeline);
-      // create container if needed
-      // TODO : should be subKeyInfo.getShouldCreateContainer(), but for now
-      //The following change has to reverted once HDFS-11888 is fixed.
-      if(!containersCreated.contains(containerName)) {
-        synchronized (containerName.intern()) {
-          //checking again, there is a chance that some other thread has
-          // created it.
-          if (!containersCreated.contains(containerName)) {
-            LOG.debug("Need to create container {}.", containerName);
-            try {
-              ContainerProtocolCalls.createContainer(xceiverClient, requestId);
-            } catch (StorageContainerException ex) {
-              if (ex.getResult().equals(Result.CONTAINER_EXISTS)) {
-                //container already exist.
-                LOG.debug("Container {} already exists.", containerName);
-              } else {
-                LOG.error("Container creation failed for {}.",
-                    containerName, ex);
-                throw ex;
-              }
-            }
-            containersCreated.add(containerName);
-          }
-        }
-      }
-
-      groupOutputStream.addStream(containerKey, keyInfo.getKeyName(),
-          xceiverClientManager, xceiverClient, requestId, chunkSize,
-          subKeyInfo.getLength());
-    }
-    return groupOutputStream;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
index 713a085..76c9be0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
@@ -19,20 +19,21 @@
 package org.apache.hadoop.ozone.web.storage;
 
 import com.google.common.base.Strings;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset
-    .LengthInputStream;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
-import org.apache.hadoop.ksm.protocolPB
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.protocolPB
     .KeySpaceManagerProtocolClientSideTranslatorPB;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneConsts.Versioning;
-import org.apache.hadoop.ozone.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
+import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
 import org.apache.hadoop.ozone.ksm.KSMConfigKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/OzoneContainerTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/OzoneContainerTranslation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/OzoneContainerTranslation.java
deleted file mode 100644
index 18ade6e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/OzoneContainerTranslation.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.web.storage;
-
-import java.util.List;
-
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.KeyData;
-import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.KeyValue;
-import org.apache.hadoop.ozone.OzoneConsts.Versioning;
-import org.apache.hadoop.ozone.web.request.OzoneQuota;
-import org.apache.hadoop.ozone.web.response.BucketInfo;
-import org.apache.hadoop.ozone.web.response.KeyInfo;
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
-import org.apache.hadoop.ozone.web.response.VolumeOwner;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * This class contains methods that define the translation between the Ozone
- * domain model and the storage container domain model.
- */
-final class OzoneContainerTranslation {
-
-  private static final String ACLS = "ACLS";
-  private static final String BUCKET = "BUCKET";
-  private static final String BUCKET_NAME = "BUCKET_NAME";
-  private static final String CREATED_BY = "CREATED_BY";
-  private static final String CREATED_ON = "CREATED_ON";
-  private static final String KEY = "KEY";
-  private static final String OWNER = "OWNER";
-  private static final String QUOTA = "QUOTA";
-  private static final String STORAGE_TYPE = "STORAGE_TYPE";
-  private static final String TYPE = "TYPE";
-  private static final String VERSIONING = "VERSIONING";
-  private static final String VOLUME = "VOLUME";
-  private static final String VOLUME_NAME = "VOLUME_NAME";
-
-  /**
-   * Creates key data intended for reading a container key.
-   *
-   * @param containerName container name
-   * @param containerKey container key
-   * @return KeyData intended for reading the container key
-   */
-  public static KeyData containerKeyDataForRead(String containerName,
-      String containerKey) {
-    return KeyData
-        .newBuilder()
-        .setContainerName(containerName)
-        .setName(containerKey)
-        .build();
-  }
-
-  /**
-   * Translates a bucket to its container representation.
-   *
-   * @param containerName container name
-   * @param containerKey container key
-   * @param bucket the bucket to translate
-   * @return KeyData representation of bucket
-   */
-  public static KeyData fromBucketToContainerKeyData(
-      String containerName, String containerKey, BucketInfo bucket) {
-    KeyData.Builder containerKeyData = KeyData
-        .newBuilder()
-        .setContainerName(containerName)
-        .setName(containerKey)
-        .addMetadata(newKeyValue(TYPE, BUCKET))
-        .addMetadata(newKeyValue(VOLUME_NAME, bucket.getVolumeName()))
-        .addMetadata(newKeyValue(BUCKET_NAME, bucket.getBucketName()));
-
-    if (bucket.getAcls() != null) {
-      containerKeyData.addMetadata(newKeyValue(ACLS,
-          StringUtils.join(',', bucket.getAcls())));
-    }
-
-    if (bucket.getVersioning() != null &&
-        bucket.getVersioning() != Versioning.NOT_DEFINED) {
-      containerKeyData.addMetadata(newKeyValue(VERSIONING,
-          bucket.getVersioning().name()));
-    }
-
-    if (bucket.getStorageType() != StorageType.RAM_DISK) {
-      containerKeyData.addMetadata(newKeyValue(STORAGE_TYPE,
-          bucket.getStorageType().name()));
-    }
-
-    return containerKeyData.build();
-  }
-
-  /**
-   * Translates a bucket from its container representation.
-   *
-   * @param metadata container metadata representing the bucket
-   * @return bucket translated from container representation
-   */
-  public static BucketInfo fromContainerKeyValueListToBucket(
-      List<KeyValue> metadata) {
-    BucketInfo bucket = new BucketInfo();
-    for (KeyValue keyValue : metadata) {
-      switch (keyValue.getKey()) {
-      case VOLUME_NAME:
-        bucket.setVolumeName(keyValue.getValue());
-        break;
-      case BUCKET_NAME:
-        bucket.setBucketName(keyValue.getValue());
-        break;
-      case VERSIONING:
-        bucket.setVersioning(
-            Enum.valueOf(Versioning.class, keyValue.getValue()));
-        break;
-      case STORAGE_TYPE:
-        bucket.setStorageType(
-            Enum.valueOf(StorageType.class, keyValue.getValue()));
-        break;
-      default:
-        break;
-      }
-    }
-    return bucket;
-  }
-
-  /**
-   * Translates a volume from its container representation.
-   *
-   * @param metadata container metadata representing the volume
-   * @return volume translated from container representation
-   */
-  public static VolumeInfo fromContainerKeyValueListToVolume(
-      List<KeyValue> metadata) {
-    VolumeInfo volume = new VolumeInfo();
-    for (KeyValue keyValue : metadata) {
-      switch (keyValue.getKey()) {
-      case VOLUME_NAME:
-        volume.setVolumeName(keyValue.getValue());
-        break;
-      case CREATED_BY:
-        volume.setCreatedBy(keyValue.getValue());
-        break;
-      case CREATED_ON:
-        volume.setCreatedOn(keyValue.getValue());
-        break;
-      case OWNER:
-        volume.setOwner(new VolumeOwner(keyValue.getValue()));
-        break;
-      case QUOTA:
-        volume.setQuota(OzoneQuota.parseQuota(keyValue.getValue()));
-        break;
-      default:
-        break;
-      }
-    }
-    return volume;
-  }
-
-  /**
-   * Translates a key to its container representation.
-   *
-   * @param containerName container name
-   * @param containerKey container key
-   * @param keyInfo key information received from call
-   * @return KeyData intended for reading the container key
-   */
-  public static KeyData fromKeyToContainerKeyData(String containerName,
-      String containerKey, KeyInfo key) {
-    return KeyData
-        .newBuilder()
-        .setContainerName(containerName)
-        .setName(containerKey)
-        .addMetadata(newKeyValue(TYPE, KEY))
-        .build();
-  }
-
-  /**
-   * Translates a key to its container representation.  The return value is a
-   * builder that can be manipulated further before building the result.
-   *
-   * @param containerName container name
-   * @param containerKey container key
-   * @param keyInfo key information received from call
-   * @return KeyData builder
-   */
-  public static KeyData.Builder fromKeyToContainerKeyDataBuilder(
-      String containerName, String containerKey, KeyInfo key) {
-    return KeyData
-        .newBuilder()
-        .setContainerName(containerName)
-        .setName(containerKey)
-        .addMetadata(newKeyValue(TYPE, KEY));
-  }
-
-  /**
-   * Translates a volume to its container representation.
-   *
-   * @param containerName container name
-   * @param containerKey container key
-   * @param volume the volume to translate
-   * @return KeyData representation of volume
-   */
-  public static KeyData fromVolumeToContainerKeyData(
-      String containerName, String containerKey, VolumeInfo volume) {
-    KeyData.Builder containerKeyData = KeyData
-        .newBuilder()
-        .setContainerName(containerName)
-        .setName(containerKey)
-        .addMetadata(newKeyValue(TYPE, VOLUME))
-        .addMetadata(newKeyValue(VOLUME_NAME, volume.getVolumeName()))
-        .addMetadata(newKeyValue(CREATED_ON, volume.getCreatedOn()));
-
-    if (volume.getQuota() != null && volume.getQuota().sizeInBytes() != -1L) {
-      containerKeyData.addMetadata(newKeyValue(QUOTA,
-          OzoneQuota.formatQuota(volume.getQuota())));
-    }
-
-    if (volume.getOwner() != null && volume.getOwner().getName() != null &&
-        !volume.getOwner().getName().isEmpty()) {
-      containerKeyData.addMetadata(newKeyValue(OWNER,
-          volume.getOwner().getName()));
-    }
-
-    if (volume.getCreatedBy() != null && !volume.getCreatedBy().isEmpty()) {
-      containerKeyData.addMetadata(
-          newKeyValue(CREATED_BY, volume.getCreatedBy()));
-    }
-
-    return containerKeyData.build();
-  }
-
-  /**
-   * Translates a key-value pair to its container representation.
-   *
-   * @param key the key
-   * @param value the value
-   * @return container representation of key-value pair
-   */
-  private static KeyValue newKeyValue(String key, Object value) {
-    return KeyValue.newBuilder().setKey(key).setValue(value.toString()).build();
-  }
-
-  /**
-   * There is no need to instantiate this class.
-   */
-  private OzoneContainerTranslation() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java
index 609a47b..6cce47e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/userauth/Simple.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.web.interfaces.UserAuth;
 import org.apache.hadoop.ozone.OzoneConsts;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
index c417601..2fe64d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java
@@ -23,11 +23,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
-import org.apache.hadoop.ozone.web.headers.Header;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.util.Time;
 
 import javax.ws.rs.core.HttpHeaders;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/Corona.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/Corona.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/Corona.java
index 581ccc6..4d69ba9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/Corona.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/Corona.java
@@ -26,7 +26,9 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.ozone.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -132,7 +134,7 @@ public final class Corona extends Configured implements Tool {
     numberOfBucketsCreated = new AtomicInteger();
     numberOfKeysAdded = new AtomicLong();
     OzoneClientFactory.setConfiguration(conf);
-    ozoneClient = OzoneClientFactory.getRpcClient();
+    ozoneClient = OzoneClientFactory.getClient();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientImpl.java
deleted file mode 100644
index b861f7d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientImpl.java
+++ /dev/null
@@ -1,357 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ozone.io.OzoneInputStream;
-import org.apache.hadoop.ozone.io.OzoneOutputStream;
-import org.apache.hadoop.ozone.web.exceptions.OzoneException;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-
-/**
- * This class is to test all the public facing APIs of Ozone Client.
- */
-public class TestOzoneClientImpl {
-
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  private static MiniOzoneCluster cluster = null;
-  private static OzoneClientImpl ozClient = null;
-
-  /**
-   * Create a MiniDFSCluster for testing.
-   * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
-   *
-   * @throws IOException
-   */
-  @BeforeClass
-  public static void init() throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
-    cluster = new MiniOzoneCluster.Builder(conf)
-        .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
-    ozClient = new OzoneClientImpl(conf);
-  }
-
-  @Test
-  public void testCreateVolume()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
-    Assert.assertEquals(volumeName, volume.getVolumeName());
-  }
-
-  @Test
-  public void testCreateVolumeWithOwner()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName, "test");
-    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
-    Assert.assertEquals(volumeName, volume.getVolumeName());
-    Assert.assertEquals("test", volume.getOwnerName());
-  }
-
-  @Test
-  public void testCreateVolumeWithQuota()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName, "test",
-        10000000000L);
-    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
-    Assert.assertEquals(volumeName, volume.getVolumeName());
-    Assert.assertEquals("test", volume.getOwnerName());
-    Assert.assertEquals(10000000000L, volume.getQuota());
-  }
-
-  @Test
-  public void testVolumeAlreadyExist()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    try {
-      ozClient.createVolume(volumeName);
-    } catch (IOException ex) {
-      Assert.assertEquals(
-          "Volume creation failed, error:VOLUME_ALREADY_EXISTS",
-          ex.getMessage());
-    }
-  }
-
-  @Test
-  public void testSetVolumeOwner()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.setVolumeOwner(volumeName, "test");
-    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
-    Assert.assertEquals("test", volume.getOwnerName());
-  }
-
-  @Test
-  public void testSetVolumeQuota()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.setVolumeQuota(volumeName, 10000000000L);
-    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
-    Assert.assertEquals(10000000000L, volume.getQuota());
-  }
-
-  @Test
-  public void testDeleteVolume()
-      throws IOException, OzoneException {
-    thrown.expectMessage("Info Volume failed, error");
-    String volumeName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    OzoneVolume volume = ozClient.getVolumeDetails(volumeName);
-    Assert.assertNotNull(volume);
-    ozClient.deleteVolume(volumeName);
-    ozClient.getVolumeDetails(volumeName);
-  }
-
-  @Test
-  public void testCreateBucket()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-  }
-
-  @Test
-  public void testCreateBucketWithVersioning()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName,
-        OzoneConsts.Versioning.ENABLED);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-    Assert.assertEquals(OzoneConsts.Versioning.ENABLED,
-        bucket.getVersioning());
-  }
-
-  @Test
-  public void testCreateBucketWithStorageType()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName, StorageType.SSD);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-    Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
-  }
-
-  @Test
-  public void testCreateBucketWithAcls()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
-        OzoneAcl.OzoneACLRights.READ_WRITE);
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName, userAcl);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-    Assert.assertTrue(bucket.getAcls().contains(userAcl));
-  }
-
-  @Test
-  public void testCreateBucketWithAllArgument()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
-        OzoneAcl.OzoneACLRights.READ_WRITE);
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName,
-        OzoneConsts.Versioning.ENABLED,
-        StorageType.SSD, userAcl);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-    Assert.assertEquals(OzoneConsts.Versioning.ENABLED,
-        bucket.getVersioning());
-    Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
-    Assert.assertTrue(bucket.getAcls().contains(userAcl));
-  }
-
-  @Test
-  public void testCreateBucketInInvalidVolume()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    try {
-      ozClient.createBucket(volumeName, bucketName);
-    } catch (IOException ex) {
-      Assert.assertEquals(
-          "Bucket creation failed, error: VOLUME_NOT_FOUND",
-          ex.getMessage());
-    }
-  }
-
-  @Test
-  public void testAddBucketAcl()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName);
-    List<OzoneAcl> acls = new ArrayList<>();
-    acls.add(new OzoneAcl(
-        OzoneAcl.OzoneACLType.USER, "test",
-        OzoneAcl.OzoneACLRights.READ_WRITE));
-    ozClient.addBucketAcls(volumeName, bucketName, acls);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-    Assert.assertTrue(bucket.getAcls().contains(acls.get(0)));
-  }
-
-  @Test
-  public void testRemoveBucketAcl()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
-        OzoneAcl.OzoneACLRights.READ_WRITE);
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName, userAcl);
-    List<OzoneAcl> acls = new ArrayList<>();
-    acls.add(userAcl);
-    ozClient.removeBucketAcls(volumeName, bucketName, acls);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-    Assert.assertTrue(!bucket.getAcls().contains(acls.get(0)));
-  }
-
-  @Test
-  public void testSetBucketVersioning()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName);
-    ozClient.setBucketVersioning(volumeName, bucketName,
-        OzoneConsts.Versioning.ENABLED);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-    Assert.assertEquals(OzoneConsts.Versioning.ENABLED,
-        bucket.getVersioning());
-  }
-
-  @Test
-  public void testSetBucketStorageType()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName);
-    ozClient.setBucketStorageType(volumeName, bucketName,
-        StorageType.SSD);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertEquals(bucketName, bucket.getBucketName());
-    Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
-  }
-
-
-  @Test
-  public void testDeleteBucket()
-      throws IOException, OzoneException {
-    thrown.expectMessage("Info Bucket failed, error");
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName);
-    OzoneBucket bucket = ozClient.getBucketDetails(volumeName, bucketName);
-    Assert.assertNotNull(bucket);
-    ozClient.deleteBucket(volumeName, bucketName);
-    ozClient.getBucketDetails(volumeName, bucketName);
-  }
-
-
-  @Test
-  public void testPutKey()
-      throws IOException, OzoneException {
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    String keyName = UUID.randomUUID().toString();
-    String value = "sample value";
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName);
-    OzoneOutputStream out = ozClient.createKey(volumeName, bucketName,
-        keyName, value.getBytes().length);
-    out.write(value.getBytes());
-    out.close();
-    OzoneKey key = ozClient.getKeyDetails(volumeName, bucketName, keyName);
-    Assert.assertEquals(keyName, key.getKeyName());
-    OzoneInputStream is = ozClient.getKey(volumeName, bucketName, keyName);
-    byte[] fileContent = new byte[value.getBytes().length];
-    is.read(fileContent);
-    Assert.assertEquals(value, new String(fileContent));
-  }
-
-  @Test
-  public void testDeleteKey()
-      throws IOException, OzoneException {
-    thrown.expectMessage("Lookup key failed, error");
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    String keyName = UUID.randomUUID().toString();
-    String value = "sample value";
-    ozClient.createVolume(volumeName);
-    ozClient.createBucket(volumeName, bucketName);
-    OzoneOutputStream out = ozClient.createKey(volumeName, bucketName,
-        keyName, value.getBytes().length);
-    out.write(value.getBytes());
-    out.close();
-    OzoneKey key = ozClient.getKeyDetails(volumeName, bucketName, keyName);
-    Assert.assertEquals(keyName, key.getKeyName());
-    ozClient.deleteKey(volumeName, bucketName, keyName);
-    ozClient.getKeyDetails(volumeName, bucketName, keyName);
-  }
-
-  /**
-   * Shutdown MiniDFSCluster.
-   */
-  @AfterClass
-  public static void shutdown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientUtils.java
deleted file mode 100644
index a5dbea7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestOzoneClientUtils.java
+++ /dev/null
@@ -1,350 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
-import org.apache.hadoop.scm.ScmConfigKeys;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import org.junit.Rule;
-import org.junit.rules.Timeout;
-
-import java.net.InetSocketAddress;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * This test class verifies the parsing of SCM endpoint config settings.
- * The parsing logic is in {@link OzoneClientUtils}.
- */
-public class TestOzoneClientUtils {
-  @Rule
-  public Timeout timeout = new Timeout(300000);
-
-  @Rule
-  public ExpectedException thrown= ExpectedException.none();
-
-  /**
-   * Verify client endpoint lookup failure if it is not configured.
-   */
-  @Test
-  public void testMissingScmClientAddress() {
-    final Configuration conf = new OzoneConfiguration();
-    thrown.expect(IllegalArgumentException.class);
-    OzoneClientUtils.getScmAddressForClients(conf);
-  }
-
-  /**
-   * Verify that the client endpoint can be correctly parsed from
-   * configuration.
-   */
-  @Test
-  public void testGetScmClientAddress() {
-    final Configuration conf = new OzoneConfiguration();
-
-    // First try a client address with just a host name. Verify it falls
-    // back to the default port.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
-    InetSocketAddress addr = OzoneClientUtils.getScmAddressForClients(conf);
-    assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
-
-    // Next try a client address with a host name and port. Verify both
-    // are used correctly.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    addr = OzoneClientUtils.getScmAddressForClients(conf);
-    assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(100));
-  }
-
-  /**
-   * Verify DataNode endpoint lookup failure if neither the client nor
-   * datanode endpoint are configured.
-   */
-  @Test
-  public void testMissingScmDataNodeAddress() {
-    final Configuration conf = new OzoneConfiguration();
-    thrown.expect(IllegalArgumentException.class);
-    OzoneClientUtils.getScmAddressForDataNodes(conf);
-  }
-
-  /**
-   * Verify that the datanode endpoint is parsed correctly.
-   * This tests the logic used by the DataNodes to determine which address
-   * to connect to.
-   */
-  @Test
-  public void testGetScmDataNodeAddress() {
-    final Configuration conf = new OzoneConfiguration();
-
-    // First try a client address with just a host name. Verify it falls
-    // back to the default port.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
-    InetSocketAddress addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
-    assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
-
-    // Next try a client address with just a host name and port. Verify the port
-    // is ignored and the default DataNode port is used.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
-    assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
-
-    // Set both OZONE_SCM_CLIENT_ADDRESS_KEY and OZONE_SCM_DATANODE_ADDRESS_KEY.
-    // Verify that the latter overrides and the port number is still the default.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "5.6.7.8");
-    addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
-    assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
-
-    // Set both OZONE_SCM_CLIENT_ADDRESS_KEY and OZONE_SCM_DATANODE_ADDRESS_KEY.
-    // Verify that the latter overrides and the port number from the latter is
-    // used.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "5.6.7.8:200");
-    addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
-    assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(200));
-  }
-
-  /**
-   * Verify that the client endpoint bind address is computed correctly.
-   * This tests the logic used by the SCM to determine its own bind address.
-   */
-  @Test
-  public void testScmClientBindHostDefault() {
-    final Configuration conf = new OzoneConfiguration();
-
-    // The bind host should be 0.0.0.0 unless OZONE_SCM_CLIENT_BIND_HOST_KEY
-    // is set differently.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
-    InetSocketAddress addr = OzoneClientUtils.getScmClientBindAddress(conf);
-    assertThat(addr.getHostString(), is("0.0.0.0"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
-
-    // The bind host should be 0.0.0.0 unless OZONE_SCM_CLIENT_BIND_HOST_KEY
-    // is set differently. The port number from OZONE_SCM_CLIENT_ADDRESS_KEY
-    // should be respected.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
-    addr = OzoneClientUtils.getScmClientBindAddress(conf);
-    assertThat(addr.getHostString(), is("0.0.0.0"));
-    assertThat(addr.getPort(), is(100));
-
-    // OZONE_SCM_CLIENT_BIND_HOST_KEY should be respected.
-    // Port number should be default if none is specified via
-    // OZONE_SCM_DATANODE_ADDRESS_KEY.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4");
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY, "5.6.7.8");
-    addr = OzoneClientUtils.getScmClientBindAddress(conf);
-    assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
-
-    // OZONE_SCM_CLIENT_BIND_HOST_KEY should be respected.
-    // Port number from OZONE_SCM_CLIENT_ADDRESS_KEY should be
-    // respected.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY, "5.6.7.8");
-    addr = OzoneClientUtils.getScmClientBindAddress(conf);
-    assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(100));
-  }
-
-  /**
-   * Verify that the DataNode endpoint bind address is computed correctly.
-   * This tests the logic used by the SCM to determine its own bind address.
-   */
-  @Test
-  public void testScmDataNodeBindHostDefault() {
-    final Configuration conf = new OzoneConfiguration();
-
-    // The bind host should be 0.0.0.0 unless OZONE_SCM_DATANODE_BIND_HOST_KEY
-    // is set differently.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
-    InetSocketAddress addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
-    assertThat(addr.getHostString(), is("0.0.0.0"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
-
-    // The bind host should be 0.0.0.0 unless OZONE_SCM_DATANODE_BIND_HOST_KEY
-    // is set differently. The port number from OZONE_SCM_DATANODE_ADDRESS_KEY
-    // should be respected.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
-    addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
-    assertThat(addr.getHostString(), is("0.0.0.0"));
-    assertThat(addr.getPort(), is(200));
-
-    // OZONE_SCM_DATANODE_BIND_HOST_KEY should be respected.
-    // Port number should be default if none is specified via
-    // OZONE_SCM_DATANODE_ADDRESS_KEY.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY, "5.6.7.8");
-    addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
-    assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
-
-    // OZONE_SCM_DATANODE_BIND_HOST_KEY should be respected.
-    // Port number from OZONE_SCM_DATANODE_ADDRESS_KEY should be
-    // respected.
-    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY, "5.6.7.8");
-    addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
-    assertThat(addr.getHostString(), is("5.6.7.8"));
-    assertThat(addr.getPort(), is(200));
-  }
-
-  @Test
-  public void testGetSCMAddresses() {
-    final Configuration conf = new OzoneConfiguration();
-    Collection<InetSocketAddress> addresses = null;
-    InetSocketAddress addr = null;
-    Iterator<InetSocketAddress> it = null;
-
-    // Verify valid IP address setup
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "1.2.3.4");
-    addresses = OzoneClientUtils.getSCMAddresses(conf);
-    assertThat(addresses.size(), is(1));
-    addr = addresses.iterator().next();
-    assertThat(addr.getHostName(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DEFAULT_PORT));
-
-    // Verify valid hostname setup
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1");
-    addresses = OzoneClientUtils.getSCMAddresses(conf);
-    assertThat(addresses.size(), is(1));
-    addr = addresses.iterator().next();
-    assertThat(addr.getHostName(), is("scm1"));
-    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DEFAULT_PORT));
-
-    // Verify valid hostname and port
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234");
-    addresses = OzoneClientUtils.getSCMAddresses(conf);
-    assertThat(addresses.size(), is(1));
-    addr = addresses.iterator().next();
-    assertThat(addr.getHostName(), is("scm1"));
-    assertThat(addr.getPort(), is(1234));
-
-    final HashMap<String, Integer> hostsAndPorts =
-        new HashMap<String, Integer>();
-    hostsAndPorts.put("scm1", 1234);
-    hostsAndPorts.put("scm2", 2345);
-    hostsAndPorts.put("scm3", 3456);
-
-    // Verify multiple hosts and port
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234,scm2:2345,scm3:3456");
-    addresses = OzoneClientUtils.getSCMAddresses(conf);
-    assertThat(addresses.size(), is(3));
-    it = addresses.iterator();
-    HashMap<String, Integer> expected1 = new HashMap<>(hostsAndPorts);
-    while(it.hasNext()) {
-      InetSocketAddress current = it.next();
-      assertTrue(expected1.remove(current.getHostName(),
-          current.getPort()));
-    }
-    assertTrue(expected1.isEmpty());
-
-    // Verify names with spaces
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, " scm1:1234, scm2:2345 , scm3:3456 ");
-    addresses = OzoneClientUtils.getSCMAddresses(conf);
-    assertThat(addresses.size(), is(3));
-    it = addresses.iterator();
-    HashMap<String, Integer> expected2 = new HashMap<>(hostsAndPorts);
-    while(it.hasNext()) {
-      InetSocketAddress current = it.next();
-      assertTrue(expected2.remove(current.getHostName(),
-          current.getPort()));
-    }
-    assertTrue(expected2.isEmpty());
-
-    // Verify empty value
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "");
-    try {
-      addresses = OzoneClientUtils.getSCMAddresses(conf);
-      fail("Empty value should cause an IllegalArgumentException");
-    } catch (Exception e) {
-      assertTrue(e instanceof IllegalArgumentException);
-    }
-
-    // Verify invalid hostname
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "s..x..:1234");
-    try {
-      addresses = OzoneClientUtils.getSCMAddresses(conf);
-      fail("An invalid hostname should cause an IllegalArgumentException");
-    } catch (Exception e) {
-      assertTrue(e instanceof IllegalArgumentException);
-    }
-
-    // Verify invalid port
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm:xyz");
-    try {
-      addresses = OzoneClientUtils.getSCMAddresses(conf);
-      fail("An invalid port should cause an IllegalArgumentException");
-    } catch (Exception e) {
-      assertTrue(e instanceof IllegalArgumentException);
-    }
-
-    // Verify a mixed case (valid and invalid value both appears)
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234, scm:xyz");
-    try {
-      addresses = OzoneClientUtils.getSCMAddresses(conf);
-      fail("An invalid value should cause an IllegalArgumentException");
-    } catch (Exception e) {
-      assertTrue(e instanceof IllegalArgumentException);
-    }
-  }
-
-  @Test
-  public void testGetKSMAddress() {
-    final Configuration conf = new OzoneConfiguration();
-
-    // First try a client address with just a host name. Verify it falls
-    // back to the default port.
-    conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "1.2.3.4");
-    InetSocketAddress addr = OzoneClientUtils.getKsmAddress(conf);
-    assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(KSMConfigKeys.OZONE_KSM_PORT_DEFAULT));
-
-    // Next try a client address with just a host name and port. Verify the port
-    // is ignored and the default KSM port is used.
-    conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "1.2.3.4:100");
-    addr = OzoneClientUtils.getKsmAddress(conf);
-    assertThat(addr.getHostString(), is("1.2.3.4"));
-    assertThat(addr.getPort(), is(100));
-
-    // Assert the we are able to use default configs if no value is specified.
-    conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "");
-    addr = OzoneClientUtils.getKsmAddress(conf);
-    assertThat(addr.getHostString(), is("0.0.0.0"));
-    assertThat(addr.getPort(), is(KSMConfigKeys.OZONE_KSM_PORT_DEFAULT));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClientUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClientUtils.java
new file mode 100644
index 0000000..4ca5d83
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClientUtils.java
@@ -0,0 +1,362 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
+import org.apache.hadoop.scm.ScmConfigKeys;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import org.junit.Rule;
+import org.junit.rules.Timeout;
+
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * This test class verifies the parsing of SCM endpoint config settings.
+ * The parsing logic is in {@link OzoneClientUtils}.
+ */
+public class TestOzoneClientUtils {
+  @Rule
+  public Timeout timeout = new Timeout(300000);
+
+  @Rule
+  public ExpectedException thrown= ExpectedException.none();
+
+  /**
+   * Verify client endpoint lookup failure if it is not configured.
+   */
+  @Test
+  public void testMissingScmClientAddress() {
+    final Configuration conf = new OzoneConfiguration();
+    thrown.expect(IllegalArgumentException.class);
+    OzoneClientUtils.getScmAddressForClients(conf);
+  }
+
+  /**
+   * Verify that the client endpoint can be correctly parsed from
+   * configuration.
+   */
+  @Test
+  public void testGetScmClientAddress() {
+    final Configuration conf = new OzoneConfiguration();
+
+    // First try a client address with just a host name. Verify it falls
+    // back to the default port.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    InetSocketAddress addr = OzoneClientUtils.getScmAddressForClients(conf);
+    assertThat(addr.getHostString(), is("1.2.3.4"));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
+
+    // Next try a client address with a host name and port. Verify both
+    // are used correctly.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    addr = OzoneClientUtils.getScmAddressForClients(conf);
+    assertThat(addr.getHostString(), is("1.2.3.4"));
+    assertThat(addr.getPort(), is(100));
+  }
+
+  /**
+   * Verify DataNode endpoint lookup failure if neither the client nor
+   * datanode endpoint are configured.
+   */
+  @Test
+  public void testMissingScmDataNodeAddress() {
+    final Configuration conf = new OzoneConfiguration();
+    thrown.expect(IllegalArgumentException.class);
+    OzoneClientUtils.getScmAddressForDataNodes(conf);
+  }
+
+  /**
+   * Verify that the datanode endpoint is parsed correctly.
+   * This tests the logic used by the DataNodes to determine which address
+   * to connect to.
+   */
+  @Test
+  public void testGetScmDataNodeAddress() {
+    final Configuration conf = new OzoneConfiguration();
+
+    // First try a client address with just a host name. Verify it falls
+    // back to the default port.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    InetSocketAddress addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
+    assertThat(addr.getHostString(), is("1.2.3.4"));
+    assertThat(addr.getPort(), is(
+        ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
+
+    // Next try a client address with just a host name and port.
+    // Verify the port is ignored and the default DataNode port is used.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
+    assertThat(addr.getHostString(), is("1.2.3.4"));
+    assertThat(addr.getPort(), is(
+        ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
+
+    // Set both OZONE_SCM_CLIENT_ADDRESS_KEY and
+    // OZONE_SCM_DATANODE_ADDRESS_KEY.
+    // Verify that the latter overrides and the port number is still the
+    // default.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "5.6.7.8");
+    addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
+    assertThat(addr.getHostString(), is("5.6.7.8"));
+    assertThat(addr.getPort(), is(
+        ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
+
+    // Set both OZONE_SCM_CLIENT_ADDRESS_KEY and
+    // OZONE_SCM_DATANODE_ADDRESS_KEY.
+    // Verify that the latter overrides and the port number from the latter is
+    // used.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "5.6.7.8:200");
+    addr = OzoneClientUtils.getScmAddressForDataNodes(conf);
+    assertThat(addr.getHostString(), is("5.6.7.8"));
+    assertThat(addr.getPort(), is(200));
+  }
+
+  /**
+   * Verify that the client endpoint bind address is computed correctly.
+   * This tests the logic used by the SCM to determine its own bind address.
+   */
+  @Test
+  public void testScmClientBindHostDefault() {
+    final Configuration conf = new OzoneConfiguration();
+
+    // The bind host should be 0.0.0.0 unless OZONE_SCM_CLIENT_BIND_HOST_KEY
+    // is set differently.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    InetSocketAddress addr = OzoneClientUtils.getScmClientBindAddress(conf);
+    assertThat(addr.getHostString(), is("0.0.0.0"));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
+
+    // The bind host should be 0.0.0.0 unless OZONE_SCM_CLIENT_BIND_HOST_KEY
+    // is set differently. The port number from OZONE_SCM_CLIENT_ADDRESS_KEY
+    // should be respected.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
+    addr = OzoneClientUtils.getScmClientBindAddress(conf);
+    assertThat(addr.getHostString(), is("0.0.0.0"));
+    assertThat(addr.getPort(), is(100));
+
+    // OZONE_SCM_CLIENT_BIND_HOST_KEY should be respected.
+    // Port number should be default if none is specified via
+    // OZONE_SCM_DATANODE_ADDRESS_KEY.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY, "5.6.7.8");
+    addr = OzoneClientUtils.getScmClientBindAddress(conf);
+    assertThat(addr.getHostString(), is("5.6.7.8"));
+    assertThat(addr.getPort(), is(
+        ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
+
+    // OZONE_SCM_CLIENT_BIND_HOST_KEY should be respected.
+    // Port number from OZONE_SCM_CLIENT_ADDRESS_KEY should be
+    // respected.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_BIND_HOST_KEY, "5.6.7.8");
+    addr = OzoneClientUtils.getScmClientBindAddress(conf);
+    assertThat(addr.getHostString(), is("5.6.7.8"));
+    assertThat(addr.getPort(), is(100));
+  }
+
+  /**
+   * Verify that the DataNode endpoint bind address is computed correctly.
+   * This tests the logic used by the SCM to determine its own bind address.
+   */
+  @Test
+  public void testScmDataNodeBindHostDefault() {
+    final Configuration conf = new OzoneConfiguration();
+
+    // The bind host should be 0.0.0.0 unless OZONE_SCM_DATANODE_BIND_HOST_KEY
+    // is set differently.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4");
+    InetSocketAddress addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
+    assertThat(addr.getHostString(), is("0.0.0.0"));
+    assertThat(addr.getPort(), is(
+        ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
+
+    // The bind host should be 0.0.0.0 unless OZONE_SCM_DATANODE_BIND_HOST_KEY
+    // is set differently. The port number from OZONE_SCM_DATANODE_ADDRESS_KEY
+    // should be respected.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
+    addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
+    assertThat(addr.getHostString(), is("0.0.0.0"));
+    assertThat(addr.getPort(), is(200));
+
+    // OZONE_SCM_DATANODE_BIND_HOST_KEY should be respected.
+    // Port number should be default if none is specified via
+    // OZONE_SCM_DATANODE_ADDRESS_KEY.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY, "5.6.7.8");
+    addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
+    assertThat(addr.getHostString(), is("5.6.7.8"));
+    assertThat(addr.getPort(), is(
+        ScmConfigKeys.OZONE_SCM_DATANODE_PORT_DEFAULT));
+
+    // OZONE_SCM_DATANODE_BIND_HOST_KEY should be respected.
+    // Port number from OZONE_SCM_DATANODE_ADDRESS_KEY should be
+    // respected.
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "1.2.3.4:100");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "1.2.3.4:200");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_BIND_HOST_KEY, "5.6.7.8");
+    addr = OzoneClientUtils.getScmDataNodeBindAddress(conf);
+    assertThat(addr.getHostString(), is("5.6.7.8"));
+    assertThat(addr.getPort(), is(200));
+  }
+
+  @Test
+  public void testGetSCMAddresses() {
+    final Configuration conf = new OzoneConfiguration();
+    Collection<InetSocketAddress> addresses = null;
+    InetSocketAddress addr = null;
+    Iterator<InetSocketAddress> it = null;
+
+    // Verify valid IP address setup
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "1.2.3.4");
+    addresses = OzoneClientUtils.getSCMAddresses(conf);
+    assertThat(addresses.size(), is(1));
+    addr = addresses.iterator().next();
+    assertThat(addr.getHostName(), is("1.2.3.4"));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DEFAULT_PORT));
+
+    // Verify valid hostname setup
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1");
+    addresses = OzoneClientUtils.getSCMAddresses(conf);
+    assertThat(addresses.size(), is(1));
+    addr = addresses.iterator().next();
+    assertThat(addr.getHostName(), is("scm1"));
+    assertThat(addr.getPort(), is(ScmConfigKeys.OZONE_SCM_DEFAULT_PORT));
+
+    // Verify valid hostname and port
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234");
+    addresses = OzoneClientUtils.getSCMAddresses(conf);
+    assertThat(addresses.size(), is(1));
+    addr = addresses.iterator().next();
+    assertThat(addr.getHostName(), is("scm1"));
+    assertThat(addr.getPort(), is(1234));
+
+    final HashMap<String, Integer> hostsAndPorts =
+        new HashMap<String, Integer>();
+    hostsAndPorts.put("scm1", 1234);
+    hostsAndPorts.put("scm2", 2345);
+    hostsAndPorts.put("scm3", 3456);
+
+    // Verify multiple hosts and port
+    conf.setStrings(
+        ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234,scm2:2345,scm3:3456");
+    addresses = OzoneClientUtils.getSCMAddresses(conf);
+    assertThat(addresses.size(), is(3));
+    it = addresses.iterator();
+    HashMap<String, Integer> expected1 = new HashMap<>(hostsAndPorts);
+    while(it.hasNext()) {
+      InetSocketAddress current = it.next();
+      assertTrue(expected1.remove(current.getHostName(),
+          current.getPort()));
+    }
+    assertTrue(expected1.isEmpty());
+
+    // Verify names with spaces
+    conf.setStrings(
+        ScmConfigKeys.OZONE_SCM_NAMES, " scm1:1234, scm2:2345 , scm3:3456 ");
+    addresses = OzoneClientUtils.getSCMAddresses(conf);
+    assertThat(addresses.size(), is(3));
+    it = addresses.iterator();
+    HashMap<String, Integer> expected2 = new HashMap<>(hostsAndPorts);
+    while(it.hasNext()) {
+      InetSocketAddress current = it.next();
+      assertTrue(expected2.remove(current.getHostName(),
+          current.getPort()));
+    }
+    assertTrue(expected2.isEmpty());
+
+    // Verify empty value
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "");
+    try {
+      addresses = OzoneClientUtils.getSCMAddresses(conf);
+      fail("Empty value should cause an IllegalArgumentException");
+    } catch (Exception e) {
+      assertTrue(e instanceof IllegalArgumentException);
+    }
+
+    // Verify invalid hostname
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "s..x..:1234");
+    try {
+      addresses = OzoneClientUtils.getSCMAddresses(conf);
+      fail("An invalid hostname should cause an IllegalArgumentException");
+    } catch (Exception e) {
+      assertTrue(e instanceof IllegalArgumentException);
+    }
+
+    // Verify invalid port
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm:xyz");
+    try {
+      addresses = OzoneClientUtils.getSCMAddresses(conf);
+      fail("An invalid port should cause an IllegalArgumentException");
+    } catch (Exception e) {
+      assertTrue(e instanceof IllegalArgumentException);
+    }
+
+    // Verify a mixed case (valid and invalid value both appears)
+    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, "scm1:1234, scm:xyz");
+    try {
+      addresses = OzoneClientUtils.getSCMAddresses(conf);
+      fail("An invalid value should cause an IllegalArgumentException");
+    } catch (Exception e) {
+      assertTrue(e instanceof IllegalArgumentException);
+    }
+  }
+
+  @Test
+  public void testGetKSMAddress() {
+    final Configuration conf = new OzoneConfiguration();
+
+    // First try a client address with just a host name. Verify it falls
+    // back to the default port.
+    conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "1.2.3.4");
+    InetSocketAddress addr = OzoneClientUtils.getKsmAddress(conf);
+    assertThat(addr.getHostString(), is("1.2.3.4"));
+    assertThat(addr.getPort(), is(KSMConfigKeys.OZONE_KSM_PORT_DEFAULT));
+
+    // Next try a client address with just a host name and port. Verify the port
+    // is ignored and the default KSM port is used.
+    conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "1.2.3.4:100");
+    addr = OzoneClientUtils.getKsmAddress(conf);
+    assertThat(addr.getHostString(), is("1.2.3.4"));
+    assertThat(addr.getPort(), is(100));
+
+    // Assert the we are able to use default configs if no value is specified.
+    conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "");
+    addr = OzoneClientUtils.getKsmAddress(conf);
+    assertThat(addr.getHostString(), is("0.0.0.0"));
+    assertThat(addr.getPort(), is(KSMConfigKeys.OZONE_KSM_PORT_DEFAULT));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/package-info.java
new file mode 100644
index 0000000..be63eab
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+/**
+ * This package contains test classes for Ozone Client.
+ */
\ No newline at end of file


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


[6/8] hadoop git commit: HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.

Posted by xy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
new file mode 100644
index 0000000..8e4ce92
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+import org.apache.hadoop.scm.XceiverClientManager;
+import org.apache.hadoop.scm.XceiverClientSpi;
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.scm.storage.ChunkInputStream;
+import org.apache.hadoop.scm.storage.ContainerProtocolCalls;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Maintaining a list of ChunkInputStream. Read based on offset.
+ */
+public class ChunkGroupInputStream extends InputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ChunkGroupInputStream.class);
+
+  private static final int EOF = -1;
+
+  private final ArrayList<ChunkInputStreamEntry> streamEntries;
+  private int currentStreamIndex;
+
+  public ChunkGroupInputStream() {
+    streamEntries = new ArrayList<>();
+    currentStreamIndex = 0;
+  }
+
+  @VisibleForTesting
+  public synchronized int getCurrentStreamIndex() {
+    return currentStreamIndex;
+  }
+
+  @VisibleForTesting
+  public long getRemainingOfIndex(int index) {
+    return streamEntries.get(index).getRemaining();
+  }
+
+  /**
+   * Append another stream to the end of the list.
+   *
+   * @param stream the stream instance.
+   * @param length the max number of bytes that should be written to this
+   *               stream.
+   */
+  public synchronized void addStream(InputStream stream, long length) {
+    streamEntries.add(new ChunkInputStreamEntry(stream, length));
+  }
+
+
+  @Override
+  public synchronized int read() throws IOException {
+    if (streamEntries.size() <= currentStreamIndex) {
+      throw new IndexOutOfBoundsException();
+    }
+    ChunkInputStreamEntry entry = streamEntries.get(currentStreamIndex);
+    int data = entry.read();
+    return data;
+  }
+
+  @Override
+  public synchronized int read(byte[] b, int off, int len) throws IOException {
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    if (off < 0 || len < 0 || len > b.length - off) {
+      throw new IndexOutOfBoundsException();
+    }
+    if (len == 0) {
+      return 0;
+    }
+    int totalReadLen = 0;
+    while (len > 0) {
+      if (streamEntries.size() <= currentStreamIndex) {
+        return totalReadLen == 0 ? EOF : totalReadLen;
+      }
+      ChunkInputStreamEntry current = streamEntries.get(currentStreamIndex);
+      int readLen = Math.min(len, (int)current.getRemaining());
+      int actualLen = current.read(b, off, readLen);
+      // this means the underlying stream has nothing at all, return
+      if (actualLen == EOF) {
+        return totalReadLen > 0? totalReadLen : EOF;
+      }
+      totalReadLen += actualLen;
+      // this means there is no more data to read beyond this point, return
+      if (actualLen != readLen) {
+        return totalReadLen;
+      }
+      off += readLen;
+      len -= readLen;
+      if (current.getRemaining() <= 0) {
+        currentStreamIndex += 1;
+      }
+    }
+    return totalReadLen;
+  }
+
+  private static class ChunkInputStreamEntry extends InputStream {
+
+    private final InputStream inputStream;
+    private final long length;
+    private long currentPosition;
+
+
+    ChunkInputStreamEntry(InputStream chunkInputStream, long length) {
+      this.inputStream = chunkInputStream;
+      this.length = length;
+      this.currentPosition = 0;
+    }
+
+    synchronized long getRemaining() {
+      return length - currentPosition;
+    }
+
+    @Override
+    public synchronized int read(byte[] b, int off, int len)
+        throws IOException {
+      int readLen = inputStream.read(b, off, len);
+      currentPosition += readLen;
+      return readLen;
+    }
+
+    @Override
+    public synchronized int read() throws IOException {
+      int data = inputStream.read();
+      currentPosition += 1;
+      return data;
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+      inputStream.close();
+    }
+  }
+
+  public static LengthInputStream getFromKsmKeyInfo(KsmKeyInfo keyInfo,
+      XceiverClientManager xceiverClientManager,
+      StorageContainerLocationProtocolClientSideTranslatorPB
+          storageContainerLocationClient, String requestId)
+      throws IOException {
+    int index = 0;
+    long length = 0;
+    String containerKey;
+    ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream();
+    for (KsmKeyLocationInfo ksmKeyLocationInfo : keyInfo.getKeyLocationList()) {
+      // check index as sanity check
+      Preconditions.checkArgument(index++ == ksmKeyLocationInfo.getIndex());
+      String containerName = ksmKeyLocationInfo.getContainerName();
+      Pipeline pipeline =
+          storageContainerLocationClient.getContainer(containerName);
+      XceiverClientSpi xceiverClient =
+          xceiverClientManager.acquireClient(pipeline);
+      boolean success = false;
+      containerKey = ksmKeyLocationInfo.getBlockID();
+      try {
+        LOG.debug("get key accessing {} {}",
+            xceiverClient.getPipeline().getContainerName(), containerKey);
+        ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation
+            .containerKeyDataForRead(
+                xceiverClient.getPipeline().getContainerName(), containerKey);
+        ContainerProtos.GetKeyResponseProto response = ContainerProtocolCalls
+            .getKey(xceiverClient, containerKeyData, requestId);
+        List<ContainerProtos.ChunkInfo> chunks =
+            response.getKeyData().getChunksList();
+        for (ContainerProtos.ChunkInfo chunk : chunks) {
+          length += chunk.getLen();
+        }
+        success = true;
+        ChunkInputStream inputStream = new ChunkInputStream(
+            containerKey, xceiverClientManager, xceiverClient,
+            chunks, requestId);
+        groupInputStream.addStream(inputStream,
+            ksmKeyLocationInfo.getLength());
+      } finally {
+        if (!success) {
+          xceiverClientManager.releaseClient(xceiverClient);
+        }
+      }
+    }
+    return new LengthInputStream(groupInputStream, length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
new file mode 100644
index 0000000..2cc12f4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
@@ -0,0 +1,327 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.Result;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+import org.apache.hadoop.scm.XceiverClientManager;
+import org.apache.hadoop.scm.XceiverClientSpi;
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.scm.container.common.helpers
+    .StorageContainerException;
+import org.apache.hadoop.scm.protocolPB
+    .StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.scm.storage.ChunkOutputStream;
+import org.apache.hadoop.scm.storage.ContainerProtocolCalls;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Maintaining a list of ChunkInputStream. Write based on offset.
+ *
+ * Note that this may write to multiple containers in one write call. In case
+ * that first container succeeded but later ones failed, the succeeded writes
+ * are not rolled back.
+ *
+ * TODO : currently not support multi-thread access.
+ */
+public class ChunkGroupOutputStream extends OutputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ChunkGroupOutputStream.class);
+
+  // array list's get(index) is O(1)
+  private final ArrayList<ChunkOutputStreamEntry> streamEntries;
+  private int currentStreamIndex;
+  private long totalSize;
+  private long byteOffset;
+
+  //This has to be removed once HDFS-11888 is resolved.
+  //local cache which will have list of created container names.
+  private static Set<String> containersCreated = new HashSet<>();
+
+  public ChunkGroupOutputStream() {
+    this.streamEntries = new ArrayList<>();
+    this.currentStreamIndex = 0;
+    this.totalSize = 0;
+    this.byteOffset = 0;
+  }
+
+  @VisibleForTesting
+  public long getByteOffset() {
+    return byteOffset;
+  }
+
+  /**
+   * Append another stream to the end of the list. Note that the streams are not
+   * actually created to this point, only enough meta data about the stream is
+   * stored. When something is to be actually written to the stream, the stream
+   * will be created (if not already).
+   *
+   * @param containerKey the key to store in the container
+   * @param key the ozone key
+   * @param xceiverClientManager xceiver manager instance
+   * @param xceiverClient xceiver manager instance
+   * @param requestID the request id
+   * @param chunkSize the chunk size for this key chunks
+   * @param length the total length of this key
+   */
+  public synchronized void addStream(String containerKey, String key,
+      XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
+      String requestID, int chunkSize, long length) {
+    streamEntries.add(new ChunkOutputStreamEntry(containerKey, key,
+        xceiverClientManager, xceiverClient, requestID, chunkSize, length));
+    totalSize += length;
+  }
+
+  @VisibleForTesting
+  public synchronized void addStream(OutputStream outputStream, long length) {
+    streamEntries.add(new ChunkOutputStreamEntry(outputStream, length));
+    totalSize += length;
+  }
+
+  @Override
+  public synchronized void write(int b) throws IOException {
+    if (streamEntries.size() <= currentStreamIndex) {
+      throw new IndexOutOfBoundsException();
+    }
+    ChunkOutputStreamEntry entry = streamEntries.get(currentStreamIndex);
+    entry.write(b);
+    if (entry.getRemaining() <= 0) {
+      currentStreamIndex += 1;
+    }
+    byteOffset += 1;
+  }
+
+  /**
+   * Try to write the bytes sequence b[off:off+len) to streams.
+   *
+   * NOTE: Throws exception if the data could not fit into the remaining space.
+   * In which case nothing will be written.
+   * TODO:May need to revisit this behaviour.
+   *
+   * @param b byte data
+   * @param off starting offset
+   * @param len length to write
+   * @throws IOException
+   */
+  @Override
+  public synchronized void write(byte[] b, int off, int len)
+      throws IOException {
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    if ((off < 0) || (off > b.length) || (len < 0) ||
+        ((off + len) > b.length) || ((off + len) < 0)) {
+      throw new IndexOutOfBoundsException();
+    }
+    if (len == 0) {
+      return;
+    }
+    if (streamEntries.size() <= currentStreamIndex) {
+      throw new IOException("Write out of stream range! stream index:" +
+          currentStreamIndex);
+    }
+    if (totalSize - byteOffset < len) {
+      throw new IOException("Can not write " + len + " bytes with only " +
+          (totalSize - byteOffset) + " byte space");
+    }
+    while (len > 0) {
+      // in theory, this condition should never violate due the check above
+      // still do a sanity check.
+      Preconditions.checkArgument(currentStreamIndex < streamEntries.size());
+      ChunkOutputStreamEntry current = streamEntries.get(currentStreamIndex);
+      int writeLen = Math.min(len, (int)current.getRemaining());
+      current.write(b, off, writeLen);
+      if (current.getRemaining() <= 0) {
+        currentStreamIndex += 1;
+      }
+      len -= writeLen;
+      off += writeLen;
+      byteOffset += writeLen;
+    }
+  }
+
+  @Override
+  public synchronized void flush() throws IOException {
+    for (int i = 0; i <= currentStreamIndex; i++) {
+      streamEntries.get(i).flush();
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    for (ChunkOutputStreamEntry entry : streamEntries) {
+      entry.close();
+    }
+  }
+
+  private static class ChunkOutputStreamEntry extends OutputStream {
+    private OutputStream outputStream;
+    private final String containerKey;
+    private final String key;
+    private final XceiverClientManager xceiverClientManager;
+    private final XceiverClientSpi xceiverClient;
+    private final String requestId;
+    private final int chunkSize;
+    // total number of bytes that should be written to this stream
+    private final long length;
+    // the current position of this stream 0 <= currentPosition < length
+    private long currentPosition;
+
+    ChunkOutputStreamEntry(String containerKey, String key,
+        XceiverClientManager xceiverClientManager,
+        XceiverClientSpi xceiverClient, String requestId, int chunkSize,
+        long length) {
+      this.outputStream = null;
+      this.containerKey = containerKey;
+      this.key = key;
+      this.xceiverClientManager = xceiverClientManager;
+      this.xceiverClient = xceiverClient;
+      this.requestId = requestId;
+      this.chunkSize = chunkSize;
+
+      this.length = length;
+      this.currentPosition = 0;
+    }
+
+    /**
+     * For testing purpose, taking a some random created stream instance.
+     * @param  outputStream a existing writable output stream
+     * @param  length the length of data to write to the stream
+     */
+    ChunkOutputStreamEntry(OutputStream outputStream, long length) {
+      this.outputStream = outputStream;
+      this.containerKey = null;
+      this.key = null;
+      this.xceiverClientManager = null;
+      this.xceiverClient = null;
+      this.requestId = null;
+      this.chunkSize = -1;
+
+      this.length = length;
+      this.currentPosition = 0;
+    }
+
+    long getLength() {
+      return length;
+    }
+
+    long getRemaining() {
+      return length - currentPosition;
+    }
+
+    private synchronized void checkStream() {
+      if (this.outputStream == null) {
+        this.outputStream = new ChunkOutputStream(containerKey,
+            key, xceiverClientManager, xceiverClient,
+            requestId, chunkSize);
+      }
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      checkStream();
+      outputStream.write(b);
+      this.currentPosition += 1;
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      checkStream();
+      outputStream.write(b, off, len);
+      this.currentPosition += len;
+    }
+
+    @Override
+    public void flush() throws IOException {
+      if (this.outputStream != null) {
+        this.outputStream.flush();
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (this.outputStream != null) {
+        this.outputStream.close();
+      }
+    }
+  }
+
+  public static ChunkGroupOutputStream getFromKsmKeyInfo(
+      KsmKeyInfo keyInfo, XceiverClientManager xceiverClientManager,
+      StorageContainerLocationProtocolClientSideTranslatorPB
+          storageContainerLocationClient,
+      int chunkSize, String requestId) throws IOException {
+    // TODO: the following createContainer and key writes may fail, in which
+    // case we should revert the above allocateKey to KSM.
+    // check index as sanity check
+    int index = 0;
+    String containerKey;
+    ChunkGroupOutputStream groupOutputStream = new ChunkGroupOutputStream();
+    for (KsmKeyLocationInfo subKeyInfo : keyInfo.getKeyLocationList()) {
+      containerKey = subKeyInfo.getBlockID();
+
+      Preconditions.checkArgument(index++ == subKeyInfo.getIndex());
+      String containerName = subKeyInfo.getContainerName();
+      Pipeline pipeline =
+          storageContainerLocationClient.getContainer(containerName);
+      XceiverClientSpi xceiverClient =
+          xceiverClientManager.acquireClient(pipeline);
+      // create container if needed
+      // TODO : should be subKeyInfo.getShouldCreateContainer(), but for now
+      //The following change has to reverted once HDFS-11888 is fixed.
+      if(!containersCreated.contains(containerName)) {
+        synchronized (containerName.intern()) {
+          //checking again, there is a chance that some other thread has
+          // created it.
+          if (!containersCreated.contains(containerName)) {
+            LOG.debug("Need to create container {}.", containerName);
+            try {
+              ContainerProtocolCalls.createContainer(xceiverClient, requestId);
+            } catch (StorageContainerException ex) {
+              if (ex.getResult().equals(Result.CONTAINER_EXISTS)) {
+                //container already exist.
+                LOG.debug("Container {} already exists.", containerName);
+              } else {
+                LOG.error("Container creation failed for {}.",
+                    containerName, ex);
+                throw ex;
+              }
+            }
+            containersCreated.add(containerName);
+          }
+        }
+      }
+
+      groupOutputStream.addStream(containerKey, keyInfo.getKeyName(),
+          xceiverClientManager, xceiverClient, requestId, chunkSize,
+          subKeyInfo.getLength());
+    }
+    return groupOutputStream;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/LengthInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/LengthInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/LengthInputStream.java
new file mode 100644
index 0000000..baf1887
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/LengthInputStream.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client.io;
+
+import java.io.FilterInputStream;
+import java.io.InputStream;
+
+/**
+ * An input stream with length.
+ */
+public class LengthInputStream extends FilterInputStream {
+
+  private final long length;
+
+  /**
+   * Create an stream.
+   * @param in the underlying input stream.
+   * @param length the length of the stream.
+   */
+  public LengthInputStream(InputStream in, long length) {
+    super(in);
+    this.length = length;
+  }
+
+  /** @return the length. */
+  public long getLength() {
+    return length;
+  }
+
+  public InputStream getWrappedStream() {
+    return in;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java
new file mode 100644
index 0000000..ca6f7aa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client.io;
+
+
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.KeyData;
+
+/**
+ * This class contains methods that define the translation between the Ozone
+ * domain model and the storage container domain model.
+ */
+final class OzoneContainerTranslation {
+
+  /**
+   * Creates key data intended for reading a container key.
+   *
+   * @param containerName container name
+   * @param containerKey container key
+   * @return KeyData intended for reading the container key
+   */
+  public static KeyData containerKeyDataForRead(String containerName,
+      String containerKey) {
+    return KeyData
+        .newBuilder()
+        .setContainerName(containerName)
+        .setName(containerKey)
+        .build();
+  }
+
+  /**
+   * There is no need to instantiate this class.
+   */
+  private OzoneContainerTranslation() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java
new file mode 100644
index 0000000..9551cdb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.hadoop.scm.storage.ChunkInputStream;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * OzoneInputStream is used to read data from Ozone.
+ * It uses SCM's {@link ChunkInputStream} for reading the data.
+ */
+public class OzoneInputStream extends InputStream {
+
+  private final ChunkGroupInputStream inputStream;
+
+  /**
+   * Constructs OzoneInputStream with ChunkInputStream.
+   *
+   * @param inputStream
+   */
+  public OzoneInputStream(ChunkGroupInputStream inputStream) {
+    this.inputStream = inputStream;
+  }
+
+  @Override
+  public int read() throws IOException {
+    return inputStream.read();
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    inputStream.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneOutputStream.java
new file mode 100644
index 0000000..5e2ad94
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneOutputStream.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.client.io;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * OzoneOutputStream is used to write data into Ozone.
+ * It uses SCM's {@link ChunkGroupOutputStream} for writing the data.
+ */
+public class OzoneOutputStream extends OutputStream {
+
+  private final ChunkGroupOutputStream outputStream;
+
+  /**
+   * Constructs OzoneOutputStream with ChunkGroupOutputStream.
+   *
+   * @param outputStream
+   */
+  public OzoneOutputStream(ChunkGroupOutputStream outputStream) {
+    this.outputStream = outputStream;
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    outputStream.write(b);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    outputStream.write(b, off, len);
+  }
+
+  @Override
+  public synchronized void flush() throws IOException {
+    outputStream.flush();
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    //commitKey can be done here, if needed.
+    outputStream.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/package-info.java
new file mode 100644
index 0000000..493ece8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client.io;
+
+/**
+ * This package contains Ozone I/O classes.
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/package-info.java
new file mode 100644
index 0000000..7e2591a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client;
+
+/**
+ * This package contains Ozone Client classes.
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/OzoneRestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/OzoneRestClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/OzoneRestClient.java
new file mode 100644
index 0000000..4955002
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/OzoneRestClient.java
@@ -0,0 +1,510 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.client.rest;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.OzoneConsts.Versioning;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+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.ksm.KSMConfigKeys;
+import org.apache.hadoop.ozone.client.rest.headers.Header;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.core.HttpHeaders;
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+
+import static java.net.HttpURLConnection.HTTP_CREATED;
+import static java.net.HttpURLConnection.HTTP_OK;
+
+/**
+ * Ozone REST Client Implementation, it connects Ozone Handler to execute
+ * client calls. This uses REST protocol for the communication with server.
+ */
+public class OzoneRestClient implements OzoneClient, Closeable {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OzoneRestClient.class);
+
+  private static final String SCHEMA = "http://";
+  private static final int DEFAULT_OZONE_PORT = 50070;
+
+  private final URI uri;
+  private final UserGroupInformation ugi;
+  private final OzoneAcl.OzoneACLRights userRights;
+  private final OzoneAcl.OzoneACLRights groupRights;
+
+
+  /**
+   * Creates OzoneRpcClient instance with new OzoneConfiguration.
+   *
+   * @throws IOException
+   */
+  public OzoneRestClient() throws IOException, URISyntaxException {
+    this(new OzoneConfiguration());
+  }
+
+   /**
+    * Creates OzoneRpcClient instance with the given configuration.
+    *
+    * @param conf
+    *
+    * @throws IOException
+    */
+  public OzoneRestClient(Configuration conf)
+      throws IOException {
+    Preconditions.checkNotNull(conf);
+    this.ugi = UserGroupInformation.getCurrentUser();
+    this.userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS,
+        KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT);
+    this.groupRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS,
+        KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS_DEFAULT);
+
+    //TODO: get uri from property ozone.reset.servers
+    URIBuilder ozoneURI = null;
+    try {
+      ozoneURI = new URIBuilder(SCHEMA + "localhost");
+      if (ozoneURI.getPort() == 0) {
+        ozoneURI.setPort(DEFAULT_OZONE_PORT);
+      }
+      uri = ozoneURI.build();
+    } catch (URISyntaxException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void createVolume(String volumeName)
+      throws IOException {
+    createVolume(volumeName, ugi.getUserName());
+  }
+
+  @Override
+  public void createVolume(String volumeName, String owner)
+      throws IOException {
+
+    createVolume(volumeName, owner, OzoneConsts.MAX_QUOTA_IN_BYTES,
+        (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createVolume(String volumeName, String owner,
+                           OzoneAcl... acls)
+      throws IOException {
+    createVolume(volumeName, owner, OzoneConsts.MAX_QUOTA_IN_BYTES, acls);
+  }
+
+  @Override
+  public void createVolume(String volumeName, String owner,
+                           long quota)
+      throws IOException {
+    createVolume(volumeName, owner, quota, (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createVolume(String volumeName, String owner,
+                           long quota, OzoneAcl... acls)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(owner);
+    Preconditions.checkNotNull(quota);
+    Preconditions.checkState(quota >= 0);
+
+    Set<OzoneAcl> aclSet = new HashSet<>();
+
+    if(acls != null) {
+      aclSet.addAll(Arrays.asList(acls));
+    }
+
+    LOG.info("Creating Volume: {}, with {} as owner and " +
+        "quota set to {} bytes.", volumeName, owner, quota);
+    HttpPost httpPost = null;
+    HttpEntity entity = null;
+    try (CloseableHttpClient httpClient = OzoneClientUtils.newHttpClient()) {
+      URIBuilder builder = new URIBuilder(uri);
+      builder.setPath("/" + volumeName);
+      String quotaString = quota + Header.OZONE_QUOTA_BYTES;
+      builder.setParameter(Header.OZONE_QUOTA_QUERY_TAG, quotaString);
+      httpPost = getHttpPost(owner, builder.build().toString());
+      for (OzoneAcl acl : aclSet) {
+        httpPost.addHeader(
+            Header.OZONE_ACLS, Header.OZONE_ACL_ADD + " " + acl.toString());
+      }
+
+      HttpResponse response = httpClient.execute(httpPost);
+      entity = response.getEntity();
+      int errorCode = response.getStatusLine().getStatusCode();
+      if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
+        return;
+      }
+      if (entity != null) {
+        throw new IOException(EntityUtils.toString(entity));
+      } else {
+        throw new IOException("Unexpected null in http payload");
+      }
+    } catch (URISyntaxException | IllegalArgumentException ex) {
+      throw new IOException(ex.getMessage());
+    } finally {
+      EntityUtils.consume(entity);
+      OzoneClientUtils.releaseConnection(httpPost);
+    }
+  }
+
+  @Override
+  public void setVolumeOwner(String volumeName, String owner)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(owner);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void setVolumeQuota(String volumeName, long quota)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(quota);
+    Preconditions.checkState(quota >= 0);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public OzoneVolume getVolumeDetails(String volumeName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public boolean checkVolumeAccess(String volumeName, OzoneAcl acl)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void deleteVolume(String volumeName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public Iterator<OzoneVolume> listVolumes(String volumePrefix)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public Iterator<OzoneVolume> listVolumes(String volumePrefix,
+                                             String user)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName)
+      throws IOException {
+    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
+        StorageType.DEFAULT, (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName,
+                           Versioning versioning)
+      throws IOException {
+    createBucket(volumeName, bucketName, versioning,
+        StorageType.DEFAULT, (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName,
+                           StorageType storageType)
+      throws IOException {
+    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
+        storageType, (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName,
+                           OzoneAcl... acls)
+      throws IOException {
+    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
+        StorageType.DEFAULT, acls);
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName,
+                           Versioning versioning, StorageType storageType,
+                           OzoneAcl... acls)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(versioning);
+    Preconditions.checkNotNull(storageType);
+
+    String owner = ugi.getUserName();
+    final List<OzoneAcl> listOfAcls = new ArrayList<>();
+
+    //User ACL
+    OzoneAcl userAcl =
+        new OzoneAcl(OzoneAcl.OzoneACLType.USER,
+            owner, userRights);
+    listOfAcls.add(userAcl);
+
+    //Group ACLs of the User
+    List<String> userGroups = Arrays.asList(UserGroupInformation
+        .createRemoteUser(owner).getGroupNames());
+    userGroups.stream().forEach((group) -> listOfAcls.add(
+        new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights)));
+
+    //ACLs passed as argument
+    if(acls != null) {
+      Arrays.stream(acls).forEach((acl) -> listOfAcls.add(acl));
+    }
+
+    LOG.info("Creating Bucket: {}/{}, with Versioning {} and " +
+        "Storage Type set to {}", volumeName, bucketName, versioning,
+        storageType);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  /**
+   * Converts OzoneConts.Versioning enum to boolean.
+   *
+   * @param version
+   * @return corresponding boolean value
+   */
+  private boolean getBucketVersioningProtobuf(
+      Versioning version) {
+    if(version != null) {
+      switch(version) {
+      case ENABLED:
+        return true;
+      case NOT_DEFINED:
+      case DISABLED:
+      default:
+        return false;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public void addBucketAcls(String volumeName, String bucketName,
+                            List<OzoneAcl> addAcls)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(addAcls);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void removeBucketAcls(String volumeName, String bucketName,
+                               List<OzoneAcl> removeAcls)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(removeAcls);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void setBucketVersioning(String volumeName, String bucketName,
+                                  Versioning versioning)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(versioning);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void setBucketStorageType(String volumeName, String bucketName,
+                                   StorageType storageType)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(storageType);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void deleteBucket(String volumeName, String bucketName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void checkBucketAccess(String volumeName, String bucketName)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public OzoneBucket getBucketDetails(String volumeName,
+                                      String bucketName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public Iterator<OzoneBucket> listBuckets(String volumeName,
+                                            String bucketPrefix)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public OzoneOutputStream createKey(String volumeName, String bucketName,
+                                     String keyName, long size)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public OzoneInputStream getKey(String volumeName, String bucketName,
+                                 String keyName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(keyName);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void deleteKey(String volumeName, String bucketName,
+                        String keyName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(keyName);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public List<OzoneKey> listKeys(String volumeName, String bucketName,
+                                 String keyPrefix)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public OzoneKey getKeyDetails(String volumeName, String bucketName,
+                                  String keyName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(keyName);
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  /**
+   * Converts Versioning to boolean.
+   *
+   * @param version
+   * @return corresponding boolean value
+   */
+  private boolean getBucketVersioningFlag(
+      Versioning version) {
+    if(version != null) {
+      switch(version) {
+      case ENABLED:
+        return true;
+      case DISABLED:
+      case NOT_DEFINED:
+      default:
+        return false;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Returns a standard HttpPost Object to use for ozone post requests.
+   *
+   * @param user - If the use is being made on behalf of user, that user
+   * @param uriString  - UriString
+   * @return HttpPost
+   */
+  public HttpPost getHttpPost(String user, String uriString) {
+    HttpPost httpPost = new HttpPost(uriString);
+    addOzoneHeaders(httpPost);
+    if (user != null) {
+      httpPost.addHeader(Header.OZONE_USER, user);
+    }
+    return httpPost;
+  }
+
+  /**
+   * Add Ozone Headers.
+   *
+   * @param httpRequest - Http Request
+   */
+  private void addOzoneHeaders(HttpRequestBase httpRequest) {
+    SimpleDateFormat format =
+        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
+
+    httpRequest.addHeader(Header.OZONE_VERSION_HEADER,
+        Header.OZONE_V1_VERSION_HEADER);
+    httpRequest.addHeader(HttpHeaders.DATE,
+        format.format(new Date(Time.monotonicNow())));
+    httpRequest.addHeader(HttpHeaders.AUTHORIZATION,
+        Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " " +
+            ugi.getUserName());
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
new file mode 100644
index 0000000..5221a0e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client.rest.headers;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * OZONE specific HTTP headers.
+ */
+@InterfaceAudience.Private
+public final class Header {
+  public static final String OZONE_QUOTA_BYTES = "BYTES";
+  public static final String OZONE_QUOTA_MB = "MB";
+  public static final String OZONE_QUOTA_GB = "GB";
+  public static final String OZONE_QUOTA_TB = "TB";
+  public static final String OZONE_QUOTA_REMOVE = "remove";
+  public static final String OZONE_QUOTA_UNDEFINED = "undefined";
+  public static final String OZONE_EMPTY_STRING="";
+  public static final String OZONE_DEFAULT_LIST_SIZE = "1000";
+
+  public static final String OZONE_USER = "x-ozone-user";
+  public static final String OZONE_SIMPLE_AUTHENTICATION_SCHEME = "OZONE";
+  public static final String OZONE_VERSION_HEADER = "x-ozone-version";
+  public static final String OZONE_V1_VERSION_HEADER ="v1";
+
+  public static final String OZONE_LIST_QUERY_SERVICE = "service";
+  public static final String OZONE_LIST_QUERY_VOLUME = "volume";
+  public static final String OZONE_LIST_QUERY_BUCKET = "bucket";
+  public static final String OZONE_LIST_QUERY_KEY = "key";
+
+  public static final String OZONE_REQUEST_ID = "x-ozone-request-id";
+  public static final String OZONE_SERVER_NAME = "x-ozone-server-name";
+
+  public static final String OZONE_STORAGE_TYPE = "x-ozone-storage-type";
+
+  public static final String OZONE_BUCKET_VERSIONING =
+      "x-ozone-bucket-versioning";
+
+  public static final String OZONE_ACLS = "x-ozone-acls";
+  public static final String OZONE_ACL_ADD = "ADD";
+  public static final String OZONE_ACL_REMOVE = "REMOVE";
+
+  public static final String OZONE_LIST_QUERY_TAG ="info";
+  public static final String OZONE_QUOTA_QUERY_TAG ="quota";
+  public static final String CONTENT_MD5 = "Content-MD5";
+  public static final String OZONE_LIST_QUERY_PREFIX="prefix";
+  public static final String OZONE_LIST_QUERY_MAXKEYS="max-keys";
+  public static final String OZONE_LIST_QUERY_PREVKEY="prev-key";
+  public static final String OZONE_LIST_QUERY_ROOTSCAN="root-scan";
+
+  private Header() {
+    // Never constructed.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/headers/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/headers/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/headers/package-info.java
new file mode 100644
index 0000000..54157f0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/headers/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Ozone HTTP header definitions.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.ozone.client.rest.headers;
+
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/package-info.java
new file mode 100644
index 0000000..ebcc104
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client.rest;
+
+/**
+ * This package contains Ozone rest client library classes.
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClient.java
new file mode 100644
index 0000000..daa9639
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClient.java
@@ -0,0 +1,578 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.client.rpc;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.protocolPB
+    .KeySpaceManagerProtocolClientSideTranslatorPB;
+import org.apache.hadoop.ozone.ksm.protocolPB
+    .KeySpaceManagerProtocolPB;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
+import org.apache.hadoop.ozone.OzoneConsts.Versioning;
+import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
+import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
+import org.apache.hadoop.scm.ScmConfigKeys;
+import org.apache.hadoop.scm.XceiverClientManager;
+import org.apache.hadoop.scm.protocolPB
+    .StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.scm.protocolPB
+    .StorageContainerLocationProtocolPB;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+/**
+ * Ozone RPC Client Implementation, it connects to KSM, SCM and DataNode
+ * to execute client calls. This uses RPC protocol for communication
+ * with the servers.
+ */
+public class OzoneRpcClient implements OzoneClient, Closeable {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OzoneRpcClient.class);
+
+  private final StorageContainerLocationProtocolClientSideTranslatorPB
+      storageContainerLocationClient;
+  private final KeySpaceManagerProtocolClientSideTranslatorPB
+      keySpaceManagerClient;
+  private final XceiverClientManager xceiverClientManager;
+  private final int chunkSize;
+
+
+  private final UserGroupInformation ugi;
+  private final OzoneAcl.OzoneACLRights userRights;
+  private final OzoneAcl.OzoneACLRights groupRights;
+
+  /**
+   * Creates OzoneRpcClient instance with new OzoneConfiguration.
+   *
+   * @throws IOException
+   */
+  public OzoneRpcClient() throws IOException {
+    this(new OzoneConfiguration());
+  }
+
+   /**
+    * Creates OzoneRpcClient instance with the given configuration.
+    *
+    * @param conf
+    *
+    * @throws IOException
+    */
+  public OzoneRpcClient(Configuration conf) throws IOException {
+    Preconditions.checkNotNull(conf);
+    this.ugi = UserGroupInformation.getCurrentUser();
+    this.userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS,
+        KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT);
+    this.groupRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS,
+        KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS_DEFAULT);
+
+    long scmVersion =
+        RPC.getProtocolVersion(StorageContainerLocationProtocolPB.class);
+    InetSocketAddress scmAddress =
+        OzoneClientUtils.getScmAddressForClients(conf);
+    RPC.setProtocolEngine(conf, StorageContainerLocationProtocolPB.class,
+        ProtobufRpcEngine.class);
+    this.storageContainerLocationClient =
+        new StorageContainerLocationProtocolClientSideTranslatorPB(
+            RPC.getProxy(StorageContainerLocationProtocolPB.class, scmVersion,
+                scmAddress, UserGroupInformation.getCurrentUser(), conf,
+                NetUtils.getDefaultSocketFactory(conf),
+                Client.getRpcTimeout(conf)));
+
+    long ksmVersion =
+        RPC.getProtocolVersion(KeySpaceManagerProtocolPB.class);
+    InetSocketAddress ksmAddress = OzoneClientUtils.getKsmAddress(conf);
+    RPC.setProtocolEngine(conf, KeySpaceManagerProtocolPB.class,
+        ProtobufRpcEngine.class);
+    this.keySpaceManagerClient =
+        new KeySpaceManagerProtocolClientSideTranslatorPB(
+            RPC.getProxy(KeySpaceManagerProtocolPB.class, ksmVersion,
+                ksmAddress, UserGroupInformation.getCurrentUser(), conf,
+                NetUtils.getDefaultSocketFactory(conf),
+                Client.getRpcTimeout(conf)));
+
+    this.xceiverClientManager = new XceiverClientManager(conf);
+
+    int configuredChunkSize = conf.getInt(
+        ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
+        ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT);
+    if(configuredChunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) {
+      LOG.warn("The chunk size ({}) is not allowed to be more than"
+              + " the maximum size ({}),"
+              + " resetting to the maximum size.",
+          configuredChunkSize, ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE);
+      chunkSize = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE;
+    } else {
+      chunkSize = configuredChunkSize;
+    }
+  }
+
+  @Override
+  public void createVolume(String volumeName)
+      throws IOException {
+    createVolume(volumeName, ugi.getUserName());
+  }
+
+  @Override
+  public void createVolume(String volumeName, String owner)
+      throws IOException {
+
+    createVolume(volumeName, owner, OzoneConsts.MAX_QUOTA_IN_BYTES,
+        (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createVolume(String volumeName, String owner,
+                           OzoneAcl... acls)
+      throws IOException {
+    createVolume(volumeName, owner, OzoneConsts.MAX_QUOTA_IN_BYTES, acls);
+  }
+
+  @Override
+  public void createVolume(String volumeName, String owner,
+                           long quota)
+      throws IOException {
+    createVolume(volumeName, owner, quota, (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createVolume(String volumeName, String owner,
+                           long quota, OzoneAcl... acls)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(owner);
+    Preconditions.checkNotNull(quota);
+    Preconditions.checkState(quota >= 0);
+    OzoneAcl userAcl =
+        new OzoneAcl(OzoneAcl.OzoneACLType.USER,
+            owner, userRights);
+    KsmVolumeArgs.Builder builder = KsmVolumeArgs.newBuilder();
+    builder.setAdminName(ugi.getUserName())
+        .setOwnerName(owner)
+        .setVolume(volumeName)
+        .setQuotaInBytes(quota)
+        .addOzoneAcls(KSMPBHelper.convertOzoneAcl(userAcl));
+
+    List<OzoneAcl> listOfAcls = new ArrayList<>();
+
+    //Group ACLs of the User
+    List<String> userGroups = Arrays.asList(UserGroupInformation
+        .createRemoteUser(owner).getGroupNames());
+    userGroups.stream().forEach((group) -> listOfAcls.add(
+        new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights)));
+
+    //ACLs passed as argument
+    if(acls != null) {
+      listOfAcls.addAll(Arrays.asList(acls));
+    }
+
+    //Remove duplicates and set
+    for (OzoneAcl ozoneAcl :
+        listOfAcls.stream().distinct().collect(Collectors.toList())) {
+      builder.addOzoneAcls(KSMPBHelper.convertOzoneAcl(ozoneAcl));
+    }
+
+    LOG.info("Creating Volume: {}, with {} as owner and quota set to {} bytes.",
+        volumeName, owner, quota);
+    keySpaceManagerClient.createVolume(builder.build());
+  }
+
+  @Override
+  public void setVolumeOwner(String volumeName, String owner)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(owner);
+    keySpaceManagerClient.setOwner(volumeName, owner);
+  }
+
+  @Override
+  public void setVolumeQuota(String volumeName, long quota)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(quota);
+    Preconditions.checkState(quota >= 0);
+    keySpaceManagerClient.setQuota(volumeName, quota);
+  }
+
+  @Override
+  public OzoneVolume getVolumeDetails(String volumeName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    KsmVolumeArgs volumeArgs =
+        keySpaceManagerClient.getVolumeInfo(volumeName);
+    return new OzoneVolume(volumeArgs);
+  }
+
+  @Override
+  public boolean checkVolumeAccess(String volumeName, OzoneAcl acl)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    return keySpaceManagerClient.checkVolumeAccess(volumeName,
+        KSMPBHelper.convertOzoneAcl(acl));
+  }
+
+  @Override
+  public void deleteVolume(String volumeName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    keySpaceManagerClient.deleteVolume(volumeName);
+  }
+
+  @Override
+  public Iterator<OzoneVolume> listVolumes(String volumePrefix)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public Iterator<OzoneVolume> listVolumes(String volumePrefix,
+                                             String user)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName)
+      throws IOException {
+    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
+        StorageType.DEFAULT, (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName,
+                           Versioning versioning)
+      throws IOException {
+    createBucket(volumeName, bucketName, versioning,
+        StorageType.DEFAULT, (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName,
+                           StorageType storageType)
+      throws IOException {
+    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
+        storageType, (OzoneAcl[])null);
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName,
+                           OzoneAcl... acls)
+      throws IOException {
+    createBucket(volumeName, bucketName, Versioning.NOT_DEFINED,
+        StorageType.DEFAULT, acls);
+  }
+
+  @Override
+  public void createBucket(String volumeName, String bucketName,
+                           Versioning versioning, StorageType storageType,
+                           OzoneAcl... acls)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(versioning);
+    Preconditions.checkNotNull(storageType);
+
+    KsmBucketInfo.Builder builder = KsmBucketInfo.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setStorageType(storageType)
+        .setIsVersionEnabled(getBucketVersioningProtobuf(
+        versioning));
+
+    String owner = ugi.getUserName();
+    final List<OzoneAcl> listOfAcls = new ArrayList<>();
+
+    //User ACL
+    OzoneAcl userAcl =
+        new OzoneAcl(OzoneAcl.OzoneACLType.USER,
+            owner, userRights);
+    listOfAcls.add(userAcl);
+
+    //Group ACLs of the User
+    List<String> userGroups = Arrays.asList(UserGroupInformation
+        .createRemoteUser(owner).getGroupNames());
+    userGroups.stream().forEach((group) -> listOfAcls.add(
+        new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights)));
+
+    //ACLs passed as argument
+    if(acls != null) {
+      Arrays.stream(acls).forEach((acl) -> listOfAcls.add(acl));
+    }
+
+    //Remove duplicates and set
+    builder.setAcls(listOfAcls.stream().distinct()
+        .collect(Collectors.toList()));
+    LOG.info("Creating Bucket: {}/{}, with Versioning {} and " +
+        "Storage Type set to {}", volumeName, bucketName, versioning,
+        storageType);
+    keySpaceManagerClient.createBucket(builder.build());
+  }
+
+  /**
+   * Converts OzoneConts.Versioning enum to boolean.
+   *
+   * @param version
+   * @return corresponding boolean value
+   */
+  private boolean getBucketVersioningProtobuf(
+      Versioning version) {
+    if(version != null) {
+      switch(version) {
+      case ENABLED:
+        return true;
+      case NOT_DEFINED:
+      case DISABLED:
+      default:
+        return false;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public void addBucketAcls(String volumeName, String bucketName,
+                            List<OzoneAcl> addAcls)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(addAcls);
+    KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setAddAcls(addAcls);
+    keySpaceManagerClient.setBucketProperty(builder.build());
+  }
+
+  @Override
+  public void removeBucketAcls(String volumeName, String bucketName,
+                               List<OzoneAcl> removeAcls)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(removeAcls);
+    KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setRemoveAcls(removeAcls);
+    keySpaceManagerClient.setBucketProperty(builder.build());
+  }
+
+  @Override
+  public void setBucketVersioning(String volumeName, String bucketName,
+                                  Versioning versioning)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(versioning);
+    KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setIsVersionEnabled(getBucketVersioningFlag(
+            versioning));
+    keySpaceManagerClient.setBucketProperty(builder.build());
+  }
+
+  @Override
+  public void setBucketStorageType(String volumeName, String bucketName,
+                                   StorageType storageType)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(storageType);
+    KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setStorageType(storageType);
+    keySpaceManagerClient.setBucketProperty(builder.build());
+  }
+
+  @Override
+  public void deleteBucket(String volumeName, String bucketName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    keySpaceManagerClient.deleteBucket(volumeName, bucketName);
+  }
+
+  @Override
+  public void checkBucketAccess(String volumeName, String bucketName)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public OzoneBucket getBucketDetails(String volumeName,
+                                      String bucketName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    KsmBucketInfo bucketInfo =
+        keySpaceManagerClient.getBucketInfo(volumeName, bucketName);
+    return new OzoneBucket(bucketInfo);
+  }
+
+  @Override
+  public Iterator<OzoneBucket> listBuckets(String volumeName,
+                                            String bucketPrefix)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public OzoneOutputStream createKey(String volumeName, String bucketName,
+                                     String keyName, long size)
+      throws IOException {
+    String requestId = UUID.randomUUID().toString();
+    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .setDataSize(size)
+        .build();
+
+    KsmKeyInfo keyInfo = keySpaceManagerClient.allocateKey(keyArgs);
+    ChunkGroupOutputStream  groupOutputStream =
+        ChunkGroupOutputStream.getFromKsmKeyInfo(keyInfo, xceiverClientManager,
+        storageContainerLocationClient, chunkSize, requestId);
+    return new OzoneOutputStream(groupOutputStream);
+  }
+
+  @Override
+  public OzoneInputStream getKey(String volumeName, String bucketName,
+                                 String keyName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(keyName);
+    String requestId = UUID.randomUUID().toString();
+    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .build();
+    KsmKeyInfo keyInfo = keySpaceManagerClient.lookupKey(keyArgs);
+    LengthInputStream lengthInputStream =
+        ChunkGroupInputStream.getFromKsmKeyInfo(
+        keyInfo, xceiverClientManager, storageContainerLocationClient,
+        requestId);
+    return new OzoneInputStream(
+        (ChunkGroupInputStream)lengthInputStream.getWrappedStream());
+  }
+
+  @Override
+  public void deleteKey(String volumeName, String bucketName,
+                        String keyName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(keyName);
+    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .build();
+    keySpaceManagerClient.deleteKey(keyArgs);
+  }
+
+  @Override
+  public List<OzoneKey> listKeys(String volumeName, String bucketName,
+                                 String keyPrefix)
+      throws IOException {
+    throw new UnsupportedOperationException("Not yet implemented.");
+  }
+
+  @Override
+  public OzoneKey getKeyDetails(String volumeName, String bucketName,
+                                  String keyName)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(keyName);
+    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .build();
+    KsmKeyInfo keyInfo =
+        keySpaceManagerClient.lookupKey(keyArgs);
+    return new OzoneKey(keyInfo);
+  }
+
+  /**
+   * Converts Versioning to boolean.
+   *
+   * @param version
+   * @return corresponding boolean value
+   */
+  private boolean getBucketVersioningFlag(
+      Versioning version) {
+    if(version != null) {
+      switch(version) {
+      case ENABLED:
+        return true;
+      case DISABLED:
+      case NOT_DEFINED:
+      default:
+        return false;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
+    IOUtils.cleanupWithLogger(LOG, keySpaceManagerClient);
+    IOUtils.cleanupWithLogger(LOG, xceiverClientManager);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/package-info.java
new file mode 100644
index 0000000..0fcc3fc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.client.rpc;
+
+/**
+ * This package contains Ozone rpc client library classes.
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
new file mode 100644
index 0000000..e69300c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/KSMConfigKeys.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.ksm;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+/**
+ * KSM Constants.
+ */
+public final class KSMConfigKeys {
+  /**
+   * Never constructed.
+   */
+  private KSMConfigKeys() {
+  }
+
+
+  public static final String OZONE_KSM_HANDLER_COUNT_KEY =
+      "ozone.ksm.handler.count.key";
+  public static final int OZONE_KSM_HANDLER_COUNT_DEFAULT = 20;
+
+  public static final String OZONE_KSM_ADDRESS_KEY =
+      "ozone.ksm.address";
+  public static final String OZONE_KSM_BIND_HOST_DEFAULT =
+      "0.0.0.0";
+  public static final int OZONE_KSM_PORT_DEFAULT = 9862;
+
+  public static final String OZONE_KSM_HTTP_ENABLED_KEY =
+      "ozone.ksm.http.enabled";
+  public static final String OZONE_KSM_HTTP_BIND_HOST_KEY =
+      "ozone.ksm.http-bind-host";
+  public static final String OZONE_KSM_HTTPS_BIND_HOST_KEY =
+      "ozone.ksm.https-bind-host";
+  public static final String OZONE_KSM_HTTP_ADDRESS_KEY =
+      "ozone.ksm.http-address";
+  public static final String OZONE_KSM_HTTPS_ADDRESS_KEY =
+      "ozone.ksm.https-address";
+  public static final String OZONE_KSM_KEYTAB_FILE =
+      "ozone.ksm.keytab.file";
+  public static final String OZONE_KSM_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
+  public static final int OZONE_KSM_HTTP_BIND_PORT_DEFAULT = 9874;
+  public static final int OZONE_KSM_HTTPS_BIND_PORT_DEFAULT = 9875;
+
+  // LevelDB cache file uses an off-heap cache in LevelDB of 128 MB.
+  public static final String OZONE_KSM_DB_CACHE_SIZE_MB =
+      "ozone.ksm.leveldb.cache.size.mb";
+  public static final int OZONE_KSM_DB_CACHE_SIZE_DEFAULT = 128;
+
+  public static final String OZONE_KSM_USER_MAX_VOLUME =
+      "ozone.ksm.user.max.volume";
+  public static final int OZONE_KSM_USER_MAX_VOLUME_DEFAULT = 1024;
+
+  // KSM Default user/group permissions
+  public static final String OZONE_KSM_USER_RIGHTS =
+      "ozone.ksm.user.rights";
+  public static final OzoneAcl.OzoneACLRights OZONE_KSM_USER_RIGHTS_DEFAULT =
+      OzoneAcl.OzoneACLRights.READ_WRITE;
+
+  public static final String OZONE_KSM_GROUP_RIGHTS =
+      "ozone.ksm.group.rights";
+  public static final OzoneAcl.OzoneACLRights OZONE_KSM_GROUP_RIGHTS_DEFAULT =
+      OzoneAcl.OzoneACLRights.READ_WRITE;
+}


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


[5/8] hadoop git commit: HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.

Posted by xy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketArgs.java
new file mode 100644
index 0000000..1211b50
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketArgs.java
@@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.helpers;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.BucketArgs;
+import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
+
+/**
+ * A class that encapsulates Bucket Arguments.
+ */
+public final class KsmBucketArgs {
+  /**
+   * Name of the volume in which the bucket belongs to.
+   */
+  private final String volumeName;
+  /**
+   * Name of the bucket.
+   */
+  private final String bucketName;
+  /**
+   * ACL's that are to be added for the bucket.
+   */
+  private List<OzoneAcl> addAcls;
+  /**
+   * ACL's that are to be removed from the bucket.
+   */
+  private List<OzoneAcl> removeAcls;
+  /**
+   * Bucket Version flag.
+   */
+  private Boolean isVersionEnabled;
+  /**
+   * Type of storage to be used for this bucket.
+   * [RAM_DISK, SSD, DISK, ARCHIVE]
+   */
+  private StorageType storageType;
+
+  /**
+   * Private constructor, constructed via builder.
+   * @param volumeName - Volume name.
+   * @param bucketName - Bucket name.
+   * @param addAcls - ACL's to be added.
+   * @param removeAcls - ACL's to be removed.
+   * @param isVersionEnabled - Bucket version flag.
+   * @param storageType - Storage type to be used.
+   */
+  private KsmBucketArgs(String volumeName, String bucketName,
+      List<OzoneAcl> addAcls, List<OzoneAcl> removeAcls,
+      Boolean isVersionEnabled, StorageType storageType) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.addAcls = addAcls;
+    this.removeAcls = removeAcls;
+    this.isVersionEnabled = isVersionEnabled;
+    this.storageType = storageType;
+  }
+
+  /**
+   * Returns the Volume Name.
+   * @return String.
+   */
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  /**
+   * Returns the Bucket Name.
+   * @return String
+   */
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  /**
+   * Returns the ACL's that are to be added.
+   * @return List<OzoneAclInfo>
+   */
+  public List<OzoneAcl> getAddAcls() {
+    return addAcls;
+  }
+
+  /**
+   * Returns the ACL's that are to be removed.
+   * @return List<OzoneAclInfo>
+   */
+  public List<OzoneAcl> getRemoveAcls() {
+    return removeAcls;
+  }
+
+  /**
+   * Returns true if bucket version is enabled, else false.
+   * @return isVersionEnabled
+   */
+  public Boolean getIsVersionEnabled() {
+    return isVersionEnabled;
+  }
+
+  /**
+   * Returns the type of storage to be used.
+   * @return StorageType
+   */
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Returns new builder class that builds a KsmBucketArgs.
+   *
+   * @return Builder
+   */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /**
+   * Builder for KsmBucketArgs.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private List<OzoneAcl> addAcls;
+    private List<OzoneAcl> removeAcls;
+    private Boolean isVersionEnabled;
+    private StorageType storageType;
+
+    public Builder setVolumeName(String volume) {
+      this.volumeName = volume;
+      return this;
+    }
+
+    public Builder setBucketName(String bucket) {
+      this.bucketName = bucket;
+      return this;
+    }
+
+    public Builder setAddAcls(List<OzoneAcl> acls) {
+      this.addAcls = acls;
+      return this;
+    }
+
+    public Builder setRemoveAcls(List<OzoneAcl> acls) {
+      this.removeAcls = acls;
+      return this;
+    }
+
+    public Builder setIsVersionEnabled(Boolean versionFlag) {
+      this.isVersionEnabled = versionFlag;
+      return this;
+    }
+
+    public Builder setStorageType(StorageType storage) {
+      this.storageType = storage;
+      return this;
+    }
+
+    /**
+     * Constructs the KsmBucketArgs.
+     * @return instance of KsmBucketArgs.
+     */
+    public KsmBucketArgs build() {
+      Preconditions.checkNotNull(volumeName);
+      Preconditions.checkNotNull(bucketName);
+      return new KsmBucketArgs(volumeName, bucketName, addAcls,
+          removeAcls, isVersionEnabled, storageType);
+    }
+  }
+
+  /**
+   * Creates BucketArgs protobuf from KsmBucketArgs.
+   */
+  public BucketArgs getProtobuf() {
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName);
+    if(addAcls != null && !addAcls.isEmpty()) {
+      builder.addAllAddAcls(addAcls.stream().map(
+          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+    }
+    if(removeAcls != null && !removeAcls.isEmpty()) {
+      builder.addAllRemoveAcls(removeAcls.stream().map(
+          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+    }
+    if(isVersionEnabled != null) {
+      builder.setIsVersionEnabled(isVersionEnabled);
+    }
+    if(storageType != null) {
+      builder.setStorageType(
+          PBHelperClient.convertStorageType(storageType));
+    }
+    return builder.build();
+  }
+
+  /**
+   * Parses BucketInfo protobuf and creates KsmBucketArgs.
+   * @param bucketArgs
+   * @return instance of KsmBucketArgs
+   */
+  public static KsmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
+    return new KsmBucketArgs(bucketArgs.getVolumeName(),
+        bucketArgs.getBucketName(),
+        bucketArgs.getAddAclsList().stream().map(
+            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
+        bucketArgs.getRemoveAclsList().stream().map(
+            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
+        bucketArgs.hasIsVersionEnabled() ?
+            bucketArgs.getIsVersionEnabled() : null,
+        bucketArgs.hasStorageType() ? PBHelperClient.convertStorageType(
+            bucketArgs.getStorageType()) : null);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketInfo.java
new file mode 100644
index 0000000..4de07aa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmBucketInfo.java
@@ -0,0 +1,212 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.BucketInfo;
+import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * A class that encapsulates Bucket Info.
+ */
+public final class KsmBucketInfo {
+  /**
+   * Name of the volume in which the bucket belongs to.
+   */
+  private final String volumeName;
+  /**
+   * Name of the bucket.
+   */
+  private final String bucketName;
+  /**
+   * ACL Information.
+   */
+  private List<OzoneAcl> acls;
+  /**
+   * Bucket Version flag.
+   */
+  private Boolean isVersionEnabled;
+  /**
+   * Type of storage to be used for this bucket.
+   * [RAM_DISK, SSD, DISK, ARCHIVE]
+   */
+  private StorageType storageType;
+
+  /**
+   * Private constructor, constructed via builder.
+   * @param volumeName - Volume name.
+   * @param bucketName - Bucket name.
+   * @param acls - list of ACLs.
+   * @param isVersionEnabled - Bucket version flag.
+   * @param storageType - Storage type to be used.
+   */
+  private KsmBucketInfo(String volumeName, String bucketName,
+                        List<OzoneAcl> acls, boolean isVersionEnabled,
+                        StorageType storageType) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.acls = acls;
+    this.isVersionEnabled = isVersionEnabled;
+    this.storageType = storageType;
+  }
+
+  /**
+   * Returns the Volume Name.
+   * @return String.
+   */
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  /**
+   * Returns the Bucket Name.
+   * @return String
+   */
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  /**
+   * Returns the ACL's associated with this bucket.
+   * @return List<OzoneAcl>
+   */
+  public List<OzoneAcl> getAcls() {
+    return acls;
+  }
+
+  /**
+   * Returns true if bucket version is enabled, else false.
+   * @return isVersionEnabled
+   */
+  public boolean getIsVersionEnabled() {
+    return isVersionEnabled;
+  }
+
+  /**
+   * Returns the type of storage to be used.
+   * @return StorageType
+   */
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Returns new builder class that builds a KsmBucketInfo.
+   *
+   * @return Builder
+   */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /**
+   * Builder for KsmBucketInfo.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private List<OzoneAcl> acls;
+    private Boolean isVersionEnabled;
+    private StorageType storageType;
+
+    Builder() {
+      //Default values
+      this.acls = new LinkedList<>();
+      this.isVersionEnabled = false;
+      this.storageType = StorageType.DISK;
+    }
+
+    public Builder setVolumeName(String volume) {
+      this.volumeName = volume;
+      return this;
+    }
+
+    public Builder setBucketName(String bucket) {
+      this.bucketName = bucket;
+      return this;
+    }
+
+    public Builder setAcls(List<OzoneAcl> listOfAcls) {
+      this.acls = listOfAcls;
+      return this;
+    }
+
+    public Builder setIsVersionEnabled(Boolean versionFlag) {
+      this.isVersionEnabled = versionFlag;
+      return this;
+    }
+
+    public Builder setStorageType(StorageType storage) {
+      this.storageType = storage;
+      return this;
+    }
+
+    /**
+     * Constructs the KsmBucketInfo.
+     * @return instance of KsmBucketInfo.
+     */
+    public KsmBucketInfo build() {
+      Preconditions.checkNotNull(volumeName);
+      Preconditions.checkNotNull(bucketName);
+      Preconditions.checkNotNull(acls);
+      Preconditions.checkNotNull(isVersionEnabled);
+      Preconditions.checkNotNull(storageType);
+      return new KsmBucketInfo(volumeName, bucketName, acls,
+          isVersionEnabled, storageType);
+    }
+  }
+
+  /**
+   * Creates BucketInfo protobuf from KsmBucketInfo.
+   */
+  public BucketInfo getProtobuf() {
+    return BucketInfo.newBuilder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .addAllAcls(acls.stream().map(
+            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()))
+        .setIsVersionEnabled(isVersionEnabled)
+        .setStorageType(PBHelperClient.convertStorageType(
+            storageType))
+        .build();
+  }
+
+  /**
+   * Parses BucketInfo protobuf and creates KsmBucketInfo.
+   * @param bucketInfo
+   * @return instance of KsmBucketInfo
+   */
+  public static KsmBucketInfo getFromProtobuf(BucketInfo bucketInfo) {
+    return new KsmBucketInfo(
+        bucketInfo.getVolumeName(),
+        bucketInfo.getBucketName(),
+        bucketInfo.getAclsList().stream().map(
+            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
+        bucketInfo.getIsVersionEnabled(),
+        PBHelperClient.convertStorageType(
+            bucketInfo.getStorageType()));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java
new file mode 100644
index 0000000..7375ec5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.helpers;
+
+/**
+ * Args for key. Client use this to specify key's attributes on  key creation
+ * (putKey()).
+ */
+public final class KsmKeyArgs {
+  private final String volumeName;
+  private final String bucketName;
+  private final String keyName;
+
+  private final long dataSize;
+
+  private KsmKeyArgs(String volumeName, String bucketName, String keyName,
+      long dataSize) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.keyName = keyName;
+    this.dataSize = dataSize;
+  }
+
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  public String getKeyName() {
+    return keyName;
+  }
+
+  public long getDataSize() {
+    return dataSize;
+  }
+
+  /**
+   * Builder class of KsmKeyArgs.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private String keyName;
+    private long dataSize;
+
+    public Builder setVolumeName(String volume) {
+      this.volumeName = volume;
+      return this;
+    }
+
+    public Builder setBucketName(String bucket) {
+      this.bucketName = bucket;
+      return this;
+    }
+
+    public Builder setKeyName(String key) {
+      this.keyName = key;
+      return this;
+    }
+
+    public Builder setDataSize(long size) {
+      this.dataSize = size;
+      return this;
+    }
+
+    public KsmKeyArgs build() {
+      return new KsmKeyArgs(volumeName, bucketName, keyName, dataSize);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java
new file mode 100644
index 0000000..6dea835
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyInfo.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyInfo;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Args for key block. The block instance for the key requested in putKey.
+ * This is returned from KSM to client, and client use class to talk to
+ * datanode. Also, this is the metadata written to ksm.db on server side.
+ */
+public final class KsmKeyInfo {
+  private final String volumeName;
+  private final String bucketName;
+  // name of key client specified
+  private final String keyName;
+  private final long dataSize;
+  private List<KsmKeyLocationInfo> keyLocationList;
+  private final long creationTime;
+  private final long modificationTime;
+
+  private KsmKeyInfo(String volumeName, String bucketName, String keyName,
+      List<KsmKeyLocationInfo> locationInfos, long dataSize, long creationTime,
+      long modificationTime) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.keyName = keyName;
+    this.dataSize = dataSize;
+    this.keyLocationList = locationInfos;
+    this.creationTime = creationTime;
+    this.modificationTime = modificationTime;
+  }
+
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  public String getKeyName() {
+    return keyName;
+  }
+
+  public long getDataSize() {
+    return dataSize;
+  }
+
+  public List<KsmKeyLocationInfo> getKeyLocationList() {
+    return keyLocationList;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  public long getModificationTime() {
+    return modificationTime;
+  }
+
+  /**
+   * Builder of KsmKeyInfo.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private String keyName;
+    private long dataSize;
+    private List<KsmKeyLocationInfo> ksmKeyLocationInfos;
+    private long creationTime;
+    private long modificationTime;
+
+    public Builder setVolumeName(String volume) {
+      this.volumeName = volume;
+      return this;
+    }
+
+    public Builder setBucketName(String bucket) {
+      this.bucketName = bucket;
+      return this;
+    }
+
+    public Builder setKeyName(String key) {
+      this.keyName = key;
+      return this;
+    }
+
+    public Builder setKsmKeyLocationInfos(
+        List<KsmKeyLocationInfo> ksmKeyLocationInfoList) {
+      this.ksmKeyLocationInfos = ksmKeyLocationInfoList;
+      return this;
+    }
+
+    public Builder setDataSize(long size) {
+      this.dataSize = size;
+      return this;
+    }
+
+    public Builder setCreationTime(long creationTime) {
+      this.creationTime = creationTime;
+      return this;
+    }
+
+    public Builder setModificationTime(long modificationTime) {
+      this.modificationTime = modificationTime;
+      return this;
+    }
+
+    public KsmKeyInfo build() {
+      return new KsmKeyInfo(
+          volumeName, bucketName, keyName, ksmKeyLocationInfos,
+          dataSize, creationTime, modificationTime);
+    }
+  }
+
+  public KeyInfo getProtobuf() {
+    return KeyInfo.newBuilder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .setDataSize(dataSize)
+        .addAllKeyLocationList(keyLocationList.stream()
+            .map(KsmKeyLocationInfo::getProtobuf).collect(Collectors.toList()))
+        .setCreationTime(creationTime)
+        .setModificationTime(modificationTime)
+        .build();
+  }
+
+  public static KsmKeyInfo getFromProtobuf(KeyInfo keyInfo) {
+    return new KsmKeyInfo(
+        keyInfo.getVolumeName(),
+        keyInfo.getBucketName(),
+        keyInfo.getKeyName(),
+        keyInfo.getKeyLocationListList().stream()
+            .map(KsmKeyLocationInfo::getFromProtobuf)
+            .collect(Collectors.toList()),
+        keyInfo.getDataSize(),
+        keyInfo.getCreationTime(),
+        keyInfo.getModificationTime());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java
new file mode 100644
index 0000000..039f8e0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.ksm.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyLocation;
+
+/**
+ * One key can be too huge to fit in one container. In which case it gets split
+ * into a number of subkeys. This class represents one such subkey instance.
+ */
+public final class KsmKeyLocationInfo {
+  private final String containerName;
+  // name of the block id SCM assigned for the key
+  private final String blockID;
+  private final boolean shouldCreateContainer;
+  // the id of this subkey in all the subkeys.
+  private final int index;
+  private final long length;
+  private final long offset;
+
+  private KsmKeyLocationInfo(String containerName,
+      String blockID, boolean shouldCreateContainer, int index,
+      long length, long offset) {
+    this.containerName = containerName;
+    this.blockID = blockID;
+    this.shouldCreateContainer = shouldCreateContainer;
+    this.index = index;
+    this.length = length;
+    this.offset = offset;
+  }
+
+  public String getContainerName() {
+    return containerName;
+  }
+
+  public String getBlockID() {
+    return blockID;
+  }
+
+  public boolean getShouldCreateContainer() {
+    return shouldCreateContainer;
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  /**
+   * Builder of KsmKeyLocationInfo.
+   */
+  public static class Builder {
+    private String containerName;
+    private String blockID;
+    private boolean shouldCreateContainer;
+    // the id of this subkey in all the subkeys.
+    private int index;
+    private long length;
+    private long offset;
+    public Builder setContainerName(String container) {
+      this.containerName = container;
+      return this;
+    }
+
+    public Builder setBlockID(String block) {
+      this.blockID = block;
+      return this;
+    }
+
+    public Builder setShouldCreateContainer(boolean create) {
+      this.shouldCreateContainer = create;
+      return this;
+    }
+
+    public Builder setIndex(int id) {
+      this.index = id;
+      return this;
+    }
+
+    public Builder setLength(long len) {
+      this.length = len;
+      return this;
+    }
+
+    public Builder setOffset(long off) {
+      this.offset = off;
+      return this;
+    }
+
+    public KsmKeyLocationInfo build() {
+      return new KsmKeyLocationInfo(containerName, blockID,
+          shouldCreateContainer, index, length, offset);
+    }
+  }
+
+  public KeyLocation getProtobuf() {
+    return KeyLocation.newBuilder()
+        .setContainerName(containerName)
+        .setBlockID(blockID)
+        .setShouldCreateContainer(shouldCreateContainer)
+        .setIndex(index)
+        .setLength(length)
+        .setOffset(offset)
+        .build();
+  }
+
+  public static KsmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) {
+    return new KsmKeyLocationInfo(
+        keyLocation.getContainerName(),
+        keyLocation.getBlockID(),
+        keyLocation.getShouldCreateContainer(),
+        keyLocation.getIndex(),
+        keyLocation.getLength(),
+        keyLocation.getOffset());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmOzoneAclMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmOzoneAclMap.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmOzoneAclMap.java
new file mode 100644
index 0000000..7d9efad
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmOzoneAclMap.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.ksm.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
+
+import java.util.List;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.HashMap;
+
+/**
+ * This helper class keeps a map of all user and their permissions.
+ */
+public class KsmOzoneAclMap {
+  // per Acl Type user:rights map
+  private ArrayList<Map<String, OzoneAclRights>> aclMaps;
+
+  KsmOzoneAclMap() {
+    aclMaps = new ArrayList<>();
+    for (OzoneAclType aclType : OzoneAclType.values()) {
+      aclMaps.add(aclType.ordinal(), new HashMap<>());
+    }
+  }
+
+  private Map<String, OzoneAclRights> getMap(OzoneAclType type) {
+    return aclMaps.get(type.ordinal());
+  }
+
+  // For a given acl type and user, get the stored acl
+  private OzoneAclRights getAcl(OzoneAclType type, String user) {
+    return getMap(type).get(user);
+  }
+
+  // Add a new acl to the map
+  public void addAcl(OzoneAclInfo acl) {
+    getMap(acl.getType()).put(acl.getName(), acl.getRights());
+  }
+
+  // for a given acl, check if the user has access rights
+  public boolean hasAccess(OzoneAclInfo acl) {
+    OzoneAclRights storedRights = getAcl(acl.getType(), acl.getName());
+    if (storedRights != null) {
+      switch (acl.getRights()) {
+      case READ:
+        return (storedRights == OzoneAclRights.READ)
+            || (storedRights == OzoneAclRights.READ_WRITE);
+      case WRITE:
+        return (storedRights == OzoneAclRights.WRITE)
+            || (storedRights == OzoneAclRights.READ_WRITE);
+      case READ_WRITE:
+        return (storedRights == OzoneAclRights.READ_WRITE);
+      default:
+        return false;
+      }
+    } else {
+      return false;
+    }
+  }
+
+  // Convert this map to OzoneAclInfo Protobuf List
+  public List<OzoneAclInfo> ozoneAclGetProtobuf() {
+    List<OzoneAclInfo> aclList = new LinkedList<>();
+    for (OzoneAclType type: OzoneAclType.values()) {
+      for (Map.Entry<String, OzoneAclRights> entry :
+          aclMaps.get(type.ordinal()).entrySet()) {
+        OzoneAclInfo aclInfo = OzoneAclInfo.newBuilder()
+            .setName(entry.getKey())
+            .setType(type)
+            .setRights(entry.getValue())
+            .build();
+        aclList.add(aclInfo);
+      }
+    }
+
+    return aclList;
+  }
+
+  // Create map from list of OzoneAclInfos
+  public static KsmOzoneAclMap ozoneAclGetFromProtobuf(
+      List<OzoneAclInfo> aclList) {
+    KsmOzoneAclMap aclMap = new KsmOzoneAclMap();
+    for (OzoneAclInfo acl : aclList) {
+      aclMap.addAcl(acl);
+    }
+    return aclMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java
new file mode 100644
index 0000000..4d60827
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.VolumeInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.KeyValue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+
+/**
+ * A class that encapsulates the KsmVolumeArgs Args.
+ */
+public final class KsmVolumeArgs {
+  private final String adminName;
+  private final String ownerName;
+  private final String volume;
+  private final long creationTime;
+  private final long quotaInBytes;
+  private final Map<String, String> keyValueMap;
+  private final KsmOzoneAclMap aclMap;
+
+  /**
+   * Private constructor, constructed via builder.
+   * @param adminName  - Administrator's name.
+   * @param ownerName  - Volume owner's name
+   * @param volume - volume name
+   * @param quotaInBytes - Volume Quota in bytes.
+   * @param keyValueMap - keyValue map.
+   * @param aclMap - User to access rights map.
+   * @param creationTime - Volume creation time.
+   */
+  private KsmVolumeArgs(String adminName, String ownerName, String volume,
+                        long quotaInBytes, Map<String, String> keyValueMap,
+                        KsmOzoneAclMap aclMap, long creationTime) {
+    this.adminName = adminName;
+    this.ownerName = ownerName;
+    this.volume = volume;
+    this.quotaInBytes = quotaInBytes;
+    this.keyValueMap = keyValueMap;
+    this.aclMap = aclMap;
+    this.creationTime = creationTime;
+  }
+
+  /**
+   * Returns the Admin Name.
+   * @return String.
+   */
+  public String getAdminName() {
+    return adminName;
+  }
+
+  /**
+   * Returns the owner Name.
+   * @return String
+   */
+  public String getOwnerName() {
+    return ownerName;
+  }
+
+  /**
+   * Returns the volume Name.
+   * @return String
+   */
+  public String getVolume() {
+    return volume;
+  }
+
+  /**
+   * Returns creation time.
+   * @return long
+   */
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  /**
+   * Returns Quota in Bytes.
+   * @return long, Quota in bytes.
+   */
+  public long getQuotaInBytes() {
+    return quotaInBytes;
+  }
+
+  public Map<String, String> getKeyValueMap() {
+    return keyValueMap;
+  }
+
+  public KsmOzoneAclMap getAclMap() {
+    return aclMap;
+  }
+  /**
+   * Returns new builder class that builds a KsmVolumeArgs.
+   *
+   * @return Builder
+   */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /**
+   * Builder for KsmVolumeArgs.
+   */
+  public static class Builder {
+    private String adminName;
+    private String ownerName;
+    private String volume;
+    private long creationTime;
+    private long quotaInBytes;
+    private Map<String, String> keyValueMap;
+    private KsmOzoneAclMap aclMap;
+
+    /**
+     * Constructs a builder.
+     */
+    Builder() {
+      keyValueMap = new HashMap<>();
+      aclMap = new KsmOzoneAclMap();
+    }
+
+    public Builder setAdminName(String adminName) {
+      this.adminName = adminName;
+      return this;
+    }
+
+    public Builder setOwnerName(String ownerName) {
+      this.ownerName = ownerName;
+      return this;
+    }
+
+    public Builder setVolume(String volume) {
+      this.volume = volume;
+      return this;
+    }
+
+    public Builder setCreationTime(long createdOn) {
+      this.creationTime = createdOn;
+      return this;
+    }
+
+    public Builder setQuotaInBytes(long quotaInBytes) {
+      this.quotaInBytes = quotaInBytes;
+      return this;
+    }
+
+    public Builder addMetadata(String key, String value) {
+      keyValueMap.put(key, value); // overwrite if present.
+      return this;
+    }
+
+    public Builder addOzoneAcls(OzoneAclInfo acl) throws IOException {
+      aclMap.addAcl(acl);
+      return this;
+    }
+
+    /**
+     * Constructs a CreateVolumeArgument.
+     * @return CreateVolumeArgs.
+     */
+    public KsmVolumeArgs build() {
+      Preconditions.checkNotNull(adminName);
+      Preconditions.checkNotNull(ownerName);
+      Preconditions.checkNotNull(volume);
+      return new KsmVolumeArgs(adminName, ownerName, volume, quotaInBytes,
+          keyValueMap, aclMap, creationTime);
+    }
+  }
+
+  public VolumeInfo getProtobuf() {
+    List<KeyValue> metadataList = new LinkedList<>();
+    for (Map.Entry<String, String> entry : keyValueMap.entrySet()) {
+      metadataList.add(KeyValue.newBuilder().setKey(entry.getKey()).
+          setValue(entry.getValue()).build());
+    }
+    List<OzoneAclInfo> aclList = aclMap.ozoneAclGetProtobuf();
+
+    return VolumeInfo.newBuilder()
+        .setAdminName(adminName)
+        .setOwnerName(ownerName)
+        .setVolume(volume)
+        .setQuotaInBytes(quotaInBytes)
+        .addAllMetadata(metadataList)
+        .addAllVolumeAcls(aclList)
+        .setCreationTime(creationTime)
+        .build();
+  }
+
+  public static KsmVolumeArgs getFromProtobuf(VolumeInfo volInfo) {
+    Map<String, String> kvMap = volInfo.getMetadataList().stream()
+        .collect(Collectors.toMap(KeyValue::getKey,
+            KeyValue::getValue));
+    KsmOzoneAclMap aclMap =
+        KsmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList());
+
+    return new KsmVolumeArgs(volInfo.getAdminName(), volInfo.getOwnerName(),
+        volInfo.getVolume(), volInfo.getQuotaInBytes(), kvMap, aclMap,
+        volInfo.getCreationTime());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java
new file mode 100644
index 0000000..1a3d486
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.helpers;
+
+import com.google.common.base.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A class that encapsulates the createVolume Args.
+ */
+public final class VolumeArgs {
+  private final String adminName;
+  private final String ownerName;
+  private final String volume;
+  private final long quotaInBytes;
+  private final Map<String, String> extendedAttributes;
+
+  /**
+   * Private constructor, constructed via builder.
+   *
+   * @param adminName - Administrator name.
+   * @param ownerName - Volume owner's name
+   * @param volume - volume name
+   * @param quotaInBytes - Volume Quota in bytes.
+   * @param keyValueMap - keyValue map.
+   */
+  private VolumeArgs(String adminName, String ownerName, String volume,
+      long quotaInBytes, Map<String, String> keyValueMap) {
+    this.adminName = adminName;
+    this.ownerName = ownerName;
+    this.volume = volume;
+    this.quotaInBytes = quotaInBytes;
+    this.extendedAttributes = keyValueMap;
+  }
+
+  /**
+   * Returns the Admin Name.
+   *
+   * @return String.
+   */
+  public String getAdminName() {
+    return adminName;
+  }
+
+  /**
+   * Returns the owner Name.
+   *
+   * @return String
+   */
+  public String getOwnerName() {
+    return ownerName;
+  }
+
+  /**
+   * Returns the volume Name.
+   *
+   * @return String
+   */
+  public String getVolume() {
+    return volume;
+  }
+
+  /**
+   * Returns Quota in Bytes.
+   *
+   * @return long, Quota in bytes.
+   */
+  public long getQuotaInBytes() {
+    return quotaInBytes;
+  }
+
+  public Map<String, String> getExtendedAttributes() {
+    return extendedAttributes;
+  }
+
+  static class Builder {
+    private String adminName;
+    private String ownerName;
+    private String volume;
+    private long quotaInBytes;
+    private Map<String, String> extendedAttributes;
+
+    /**
+     * Constructs a builder.
+     */
+    Builder() {
+      extendedAttributes = new HashMap<>();
+    }
+
+    public void setAdminName(String adminName) {
+      this.adminName = adminName;
+    }
+
+    public void setOwnerName(String ownerName) {
+      this.ownerName = ownerName;
+    }
+
+    public void setVolume(String volume) {
+      this.volume = volume;
+    }
+
+    public void setQuotaInBytes(long quotaInBytes) {
+      this.quotaInBytes = quotaInBytes;
+    }
+
+    public void addMetadata(String key, String value) {
+      extendedAttributes.put(key, value); // overwrite if present.
+    }
+
+    /**
+     * Constructs a CreateVolumeArgument.
+     *
+     * @return CreateVolumeArgs.
+     */
+    public VolumeArgs build() {
+      Preconditions.checkNotNull(adminName);
+      Preconditions.checkNotNull(ownerName);
+      Preconditions.checkNotNull(volume);
+      return new VolumeArgs(adminName, ownerName, volume, quotaInBytes,
+          extendedAttributes);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java
new file mode 100644
index 0000000..ce627a5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.helpers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java
new file mode 100644
index 0000000..7698ee1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm;
+/**
+ This package contains client side protocol library to communicate with KSM.
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java
new file mode 100644
index 0000000..7d84114
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java
@@ -0,0 +1,212 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.protocol;
+
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol to talk to KSM.
+ */
+public interface KeySpaceManagerProtocol {
+
+  /**
+   * Creates a volume.
+   * @param args - Arguments to create Volume.
+   * @throws IOException
+   */
+  void createVolume(KsmVolumeArgs args) throws IOException;
+
+  /**
+   * Changes the owner of a volume.
+   * @param volume  - Name of the volume.
+   * @param owner - Name of the owner.
+   * @throws IOException
+   */
+  void setOwner(String volume, String owner) throws IOException;
+
+  /**
+   * Changes the Quota on a volume.
+   * @param volume - Name of the volume.
+   * @param quota - Quota in bytes.
+   * @throws IOException
+   */
+  void setQuota(String volume, long quota) throws IOException;
+
+  /**
+   * Checks if the specified user can access this volume.
+   * @param volume - volume
+   * @param userAcl - user acls which needs to be checked for access
+   * @return true if the user has required access for the volume,
+   *         false otherwise
+   * @throws IOException
+   */
+  boolean checkVolumeAccess(String volume, OzoneAclInfo userAcl)
+      throws IOException;
+
+  /**
+   * Gets the volume information.
+   * @param volume - Volume name.
+   * @return VolumeArgs or exception is thrown.
+   * @throws IOException
+   */
+  KsmVolumeArgs getVolumeInfo(String volume) throws IOException;
+
+  /**
+   * Deletes an existing empty volume.
+   * @param volume - Name of the volume.
+   * @throws IOException
+   */
+  void deleteVolume(String volume) throws IOException;
+
+  /**
+   * Lists volume owned by a specific user.
+   * @param userName - user name
+   * @param prefix  - Filter prefix -- Return only entries that match this.
+   * @param prevKey - Previous key -- List starts from the next from the prevkey
+   * @param maxKeys - Max number of keys to return.
+   * @return List of Volumes.
+   * @throws IOException
+   */
+  List<KsmVolumeArgs> listVolumeByUser(String userName, String prefix, String
+      prevKey, int maxKeys) throws IOException;
+
+  /**
+   * Lists volume all volumes in the cluster.
+   * @param prefix  - Filter prefix -- Return only entries that match this.
+   * @param prevKey - Previous key -- List starts from the next from the prevkey
+   * @param maxKeys - Max number of keys to return.
+   * @return List of Volumes.
+   * @throws IOException
+   */
+  List<KsmVolumeArgs> listAllVolumes(String prefix, String
+      prevKey, int maxKeys) throws IOException;
+
+  /**
+   * Creates a bucket.
+   * @param bucketInfo - BucketInfo to create Bucket.
+   * @throws IOException
+   */
+  void createBucket(KsmBucketInfo bucketInfo) throws IOException;
+
+  /**
+   * Gets the bucket information.
+   * @param volumeName - Volume name.
+   * @param bucketName - Bucket name.
+   * @return KsmBucketInfo or exception is thrown.
+   * @throws IOException
+   */
+  KsmBucketInfo getBucketInfo(String volumeName, String bucketName)
+      throws IOException;
+
+  /**
+   * Sets bucket property from args.
+   * @param args - BucketArgs.
+   * @throws IOException
+   */
+  void setBucketProperty(KsmBucketArgs args) throws IOException;
+
+  /**
+   * Allocate a block to a container, the block is returned to the client.
+   *
+   * @param args the args of the key.
+   * @return KsmKeyInfo isntacne that client uses to talk to container.
+   * @throws IOException
+   */
+  KsmKeyInfo allocateKey(KsmKeyArgs args) throws IOException;
+
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return KsmKeyInfo isntacne that client uses to talk to container.
+   * @throws IOException
+   */
+  KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException;
+
+  /**
+   * Deletes an existing key.
+   *
+   * @param args the args of the key.
+   * @throws IOException
+   */
+  void deleteKey(KsmKeyArgs args) throws IOException;
+
+  /**
+   * Deletes an existing empty bucket from volume.
+   * @param volume - Name of the volume.
+   * @param bucket - Name of the bucket.
+   * @throws IOException
+   */
+  void deleteBucket(String volume, String bucket) throws IOException;
+
+  /**
+   * Returns a list of buckets represented by {@link KsmBucketInfo}
+   * in the given volume. Argument volumeName is required, others
+   * are optional.
+   *
+   * @param volumeName
+   *   the name of the volume.
+   * @param startBucketName
+   *   the start bucket name, only the buckets whose name is
+   *   after this value will be included in the result.
+   * @param bucketPrefix
+   *   bucket name prefix, only the buckets whose name has
+   *   this prefix will be included in the result.
+   * @param maxNumOfBuckets
+   *   the maximum number of buckets to return. It ensures
+   *   the size of the result will not exceed this limit.
+   * @return a list of buckets.
+   * @throws IOException
+   */
+  List<KsmBucketInfo> listBuckets(String volumeName,
+      String startBucketName, String bucketPrefix, int maxNumOfBuckets)
+      throws IOException;
+
+  /**
+   * Returns a list of keys represented by {@link KsmKeyInfo}
+   * in the given bucket. Argument volumeName, bucketName is required,
+   * others are optional.
+   *
+   * @param volumeName
+   *   the name of the volume.
+   * @param bucketName
+   *   the name of the bucket.
+   * @param startKeyName
+   *   the start key name, only the keys whose name is
+   *   after this value will be included in the result.
+   * @param keyPrefix
+   *   key name prefix, only the keys whose name has
+   *   this prefix will be included in the result.
+   * @param maxKeys
+   *   the maximum number of keys to return. It ensures
+   *   the size of the result will not exceed this limit.
+   * @return a list of keys.
+   * @throws IOException
+   */
+  List<KsmKeyInfo> listKeys(String volumeName,
+      String bucketName, String startKeyName, String keyPrefix, int maxKeys)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java
new file mode 100644
index 0000000..f77e5fd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java
@@ -0,0 +1,19 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.ksm.protocol;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000..a13c415
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
@@ -0,0 +1,651 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.protocolPB;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolTranslator;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
+import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
+import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.BucketArgs;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.BucketInfo;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.CreateBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.CreateBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.InfoBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.InfoBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.SetBucketPropertyRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.SetBucketPropertyResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.DeleteBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.DeleteBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.CreateVolumeRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.CreateVolumeResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.LocateKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.LocateKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.SetVolumePropertyRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.SetVolumePropertyResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.DeleteVolumeRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.DeleteVolumeResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.InfoVolumeRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.InfoVolumeResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.CheckVolumeAccessRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.CheckVolumeAccessResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.ListBucketsRequest;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.ListBucketsResponse;
+import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.ListKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.ListKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.VolumeInfo;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.Status;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
+import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
+    .ListVolumeRequest;
+import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
+    .ListVolumeResponse;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.stream.Collectors;
+
+/**
+ *  The client side implementation of KeySpaceManagerProtocol.
+ */
+
+@InterfaceAudience.Private
+public final class KeySpaceManagerProtocolClientSideTranslatorPB
+    implements KeySpaceManagerProtocol, ProtocolTranslator, Closeable {
+
+  /**
+   * RpcController is not used and hence is set to null.
+   */
+  private static final RpcController NULL_RPC_CONTROLLER = null;
+
+  private final KeySpaceManagerProtocolPB rpcProxy;
+
+  /**
+   * Constructor for KeySpaceManger Client.
+   * @param rpcProxy
+   */
+  public KeySpaceManagerProtocolClientSideTranslatorPB(
+      KeySpaceManagerProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
+  }
+
+  /**
+   * Closes this stream and releases any system resources associated
+   * with it. If the stream is already closed then invoking this
+   * method has no effect.
+   * <p>
+   * <p> As noted in {@link AutoCloseable#close()}, cases where the
+   * close may fail require careful attention. It is strongly advised
+   * to relinquish the underlying resources and to internally
+   * <em>mark</em> the {@code Closeable} as closed, prior to throwing
+   * the {@code IOException}.
+   *
+   * @throws IOException if an I/O error occurs
+   */
+  @Override
+  public void close() throws IOException {
+
+  }
+
+  /**
+   * Creates a volume.
+   *
+   * @param args - Arguments to create Volume.
+   * @throws IOException
+   */
+  @Override
+  public void createVolume(KsmVolumeArgs args) throws IOException {
+    CreateVolumeRequest.Builder req =
+        CreateVolumeRequest.newBuilder();
+    VolumeInfo volumeInfo = args.getProtobuf();
+    req.setVolumeInfo(volumeInfo);
+
+    final CreateVolumeResponse resp;
+    try {
+      resp = rpcProxy.createVolume(NULL_RPC_CONTROLLER,
+          req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+    if (resp.getStatus() != Status.OK) {
+      throw new
+          IOException("Volume creation failed, error:" + resp.getStatus());
+    }
+  }
+
+  /**
+   * Changes the owner of a volume.
+   *
+   * @param volume - Name of the volume.
+   * @param owner - Name of the owner.
+   * @throws IOException
+   */
+  @Override
+  public void setOwner(String volume, String owner) throws IOException {
+    SetVolumePropertyRequest.Builder req =
+        SetVolumePropertyRequest.newBuilder();
+    req.setVolumeName(volume).setOwnerName(owner);
+    final SetVolumePropertyResponse resp;
+    try {
+      resp = rpcProxy.setVolumeProperty(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new
+          IOException("Volume owner change failed, error:" + resp.getStatus());
+    }
+  }
+
+  /**
+   * Changes the Quota on a volume.
+   *
+   * @param volume - Name of the volume.
+   * @param quota - Quota in bytes.
+   * @throws IOException
+   */
+  @Override
+  public void setQuota(String volume, long quota) throws IOException {
+    SetVolumePropertyRequest.Builder req =
+        SetVolumePropertyRequest.newBuilder();
+    req.setVolumeName(volume).setQuotaInBytes(quota);
+    final SetVolumePropertyResponse resp;
+    try {
+      resp = rpcProxy.setVolumeProperty(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new
+          IOException("Volume quota change failed, error:" + resp.getStatus());
+    }
+  }
+
+  /**
+   * Checks if the specified user can access this volume.
+   *
+   * @param volume - volume
+   * @param userAcl - user acls which needs to be checked for access
+   * @return true if the user has required access for the volume,
+   *         false otherwise
+   * @throws IOException
+   */
+  @Override
+  public boolean checkVolumeAccess(String volume, OzoneAclInfo userAcl) throws
+      IOException {
+    CheckVolumeAccessRequest.Builder req =
+        CheckVolumeAccessRequest.newBuilder();
+    req.setVolumeName(volume).setUserAcl(userAcl);
+    final CheckVolumeAccessResponse resp;
+    try {
+      resp = rpcProxy.checkVolumeAccess(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+    if (resp.getStatus() == Status.ACCESS_DENIED) {
+      return false;
+    } else if (resp.getStatus() == Status.OK) {
+      return true;
+    } else {
+      throw new
+          IOException("Check Volume Access failed, error:" + resp.getStatus());
+    }
+  }
+
+  /**
+   * Gets the volume information.
+   *
+   * @param volume - Volume name.
+   * @return KsmVolumeArgs or exception is thrown.
+   * @throws IOException
+   */
+  @Override
+  public KsmVolumeArgs getVolumeInfo(String volume) throws IOException {
+    InfoVolumeRequest.Builder req = InfoVolumeRequest.newBuilder();
+    req.setVolumeName(volume);
+    final InfoVolumeResponse resp;
+    try {
+      resp = rpcProxy.infoVolume(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new
+          IOException("Info Volume failed, error:" + resp.getStatus());
+    }
+    return KsmVolumeArgs.getFromProtobuf(resp.getVolumeInfo());
+  }
+
+  /**
+   * Deletes an existing empty volume.
+   *
+   * @param volume - Name of the volume.
+   * @throws IOException
+   */
+  @Override
+  public void deleteVolume(String volume) throws IOException {
+    DeleteVolumeRequest.Builder req = DeleteVolumeRequest.newBuilder();
+    req.setVolumeName(volume);
+    final DeleteVolumeResponse resp;
+    try {
+      resp = rpcProxy.deleteVolume(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new
+          IOException("Delete Volume failed, error:" + resp.getStatus());
+    }
+  }
+
+  /**
+   * Lists volume owned by a specific user.
+   *
+   * @param userName - user name
+   * @param prefix - Filter prefix -- Return only entries that match this.
+   * @param prevKey - Previous key -- List starts from the next from the
+   * prevkey
+   * @param maxKeys - Max number of keys to return.
+   * @return List of Volumes.
+   * @throws IOException
+   */
+  @Override
+  public List<KsmVolumeArgs> listVolumeByUser(String userName, String prefix,
+                                              String prevKey, int maxKeys)
+      throws IOException {
+    ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
+    if (!Strings.isNullOrEmpty(prefix)) {
+      builder.setPrefix(prefix);
+    }
+    if (!Strings.isNullOrEmpty(prevKey)) {
+      builder.setPrevKey(prevKey);
+    }
+    builder.setMaxKeys(maxKeys);
+    builder.setUserName(userName);
+    builder.setScope(ListVolumeRequest.Scope.VOLUMES_BY_USER);
+    return listVolume(builder.build());
+  }
+
+  /**
+   * Lists volume all volumes in the cluster.
+   *
+   * @param prefix - Filter prefix -- Return only entries that match this.
+   * @param prevKey - Previous key -- List starts from the next from the
+   * prevkey
+   * @param maxKeys - Max number of keys to return.
+   * @return List of Volumes.
+   * @throws IOException
+   */
+  @Override
+  public List<KsmVolumeArgs> listAllVolumes(String prefix, String prevKey,
+      int maxKeys) throws IOException {
+    ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
+    if (!Strings.isNullOrEmpty(prefix)) {
+      builder.setPrefix(prefix);
+    }
+    if (!Strings.isNullOrEmpty(prevKey)) {
+      builder.setPrevKey(prevKey);
+    }
+    builder.setMaxKeys(maxKeys);
+    builder.setScope(ListVolumeRequest.Scope.VOLUMES_BY_CLUSTER);
+    return listVolume(builder.build());
+  }
+
+  private List<KsmVolumeArgs> listVolume(ListVolumeRequest request)
+      throws IOException {
+    final ListVolumeResponse resp;
+    try {
+      resp = rpcProxy.listVolumes(NULL_RPC_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+    if (resp.getStatus() != Status.OK) {
+      throw new IOException("List volume failed, error: "
+          + resp.getStatus());
+    }
+
+    List<KsmVolumeArgs> result = Lists.newArrayList();
+    for (VolumeInfo volInfo : resp.getVolumeInfoList()) {
+      KsmVolumeArgs volArgs = KsmVolumeArgs.getFromProtobuf(volInfo);
+      result.add(volArgs);
+    }
+
+    return resp.getVolumeInfoList().stream()
+        .map(item -> KsmVolumeArgs.getFromProtobuf(item))
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * Creates a bucket.
+   *
+   * @param bucketInfo - BucketInfo to create bucket.
+   * @throws IOException
+   */
+  @Override
+  public void createBucket(KsmBucketInfo bucketInfo) throws IOException {
+    CreateBucketRequest.Builder req =
+        CreateBucketRequest.newBuilder();
+    BucketInfo bucketInfoProtobuf = bucketInfo.getProtobuf();
+    req.setBucketInfo(bucketInfoProtobuf);
+
+    final CreateBucketResponse resp;
+    try {
+      resp = rpcProxy.createBucket(NULL_RPC_CONTROLLER,
+          req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new IOException("Bucket creation failed, error: "
+          + resp.getStatus());
+    }
+  }
+
+  /**
+   * Gets the bucket information.
+   *
+   * @param volume - Volume name.
+   * @param bucket - Bucket name.
+   * @return KsmBucketInfo or exception is thrown.
+   * @throws IOException
+   */
+  @Override
+  public KsmBucketInfo getBucketInfo(String volume, String bucket)
+      throws IOException {
+    InfoBucketRequest.Builder req =
+        InfoBucketRequest.newBuilder();
+    req.setVolumeName(volume);
+    req.setBucketName(bucket);
+
+    final InfoBucketResponse resp;
+    try {
+      resp = rpcProxy.infoBucket(NULL_RPC_CONTROLLER,
+          req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() == Status.OK) {
+      return KsmBucketInfo.getFromProtobuf(resp.getBucketInfo());
+    } else {
+      throw new IOException("Info Bucket failed, error: "
+          + resp.getStatus());
+    }
+  }
+
+  /**
+   * Sets bucket property from args.
+   * @param args - BucketArgs.
+   * @throws IOException
+   */
+  @Override
+  public void setBucketProperty(KsmBucketArgs args)
+      throws IOException {
+    SetBucketPropertyRequest.Builder req =
+        SetBucketPropertyRequest.newBuilder();
+    BucketArgs bucketArgs = args.getProtobuf();
+    req.setBucketArgs(bucketArgs);
+    final SetBucketPropertyResponse resp;
+    try {
+      resp = rpcProxy.setBucketProperty(NULL_RPC_CONTROLLER,
+          req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new IOException("Setting bucket property failed, error: "
+          + resp.getStatus());
+    }
+  }
+
+  /**
+   * List buckets in a volume.
+   *
+   * @param volumeName
+   * @param startKey
+   * @param prefix
+   * @param count
+   * @return
+   * @throws IOException
+   */
+  @Override
+  public List<KsmBucketInfo> listBuckets(String volumeName,
+      String startKey, String prefix, int count) throws IOException {
+    List<KsmBucketInfo> buckets = new ArrayList<>();
+    ListBucketsRequest.Builder reqBuilder = ListBucketsRequest.newBuilder();
+    reqBuilder.setVolumeName(volumeName);
+    reqBuilder.setCount(count);
+    if (startKey != null) {
+      reqBuilder.setStartKey(startKey);
+    }
+    if (prefix != null) {
+      reqBuilder.setPrefix(prefix);
+    }
+    ListBucketsRequest request = reqBuilder.build();
+    final ListBucketsResponse resp;
+    try {
+      resp = rpcProxy.listBuckets(NULL_RPC_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+    if (resp.getStatus() == Status.OK) {
+      buckets.addAll(
+          resp.getBucketInfoList().stream()
+              .map(KsmBucketInfo::getFromProtobuf)
+              .collect(Collectors.toList()));
+      return buckets;
+    } else {
+      throw new IOException("List Buckets failed, error: "
+          + resp.getStatus());
+    }
+  }
+
+  /**
+   * Allocate a block for a key, then use the returned meta info to talk to data
+   * node to actually write the key.
+   * @param args the args for the key to be allocated
+   * @return a handler to the key, returned client
+   * @throws IOException
+   */
+  @Override
+  public KsmKeyInfo allocateKey(KsmKeyArgs args) throws IOException {
+    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
+    KeyArgs keyArgs = KeyArgs.newBuilder()
+        .setVolumeName(args.getVolumeName())
+        .setBucketName(args.getBucketName())
+        .setKeyName(args.getKeyName())
+        .setDataSize(args.getDataSize()).build();
+    req.setKeyArgs(keyArgs);
+
+    final LocateKeyResponse resp;
+    try {
+      resp = rpcProxy.createKey(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new IOException("Create key failed, error:" +
+          resp.getStatus());
+    }
+    return KsmKeyInfo.getFromProtobuf(resp.getKeyInfo());
+  }
+
+  @Override
+  public KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException {
+    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
+    KeyArgs keyArgs = KeyArgs.newBuilder()
+        .setVolumeName(args.getVolumeName())
+        .setBucketName(args.getBucketName())
+        .setKeyName(args.getKeyName())
+        .setDataSize(args.getDataSize()).build();
+    req.setKeyArgs(keyArgs);
+
+    final LocateKeyResponse resp;
+    try {
+      resp = rpcProxy.lookupKey(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new IOException("Lookup key failed, error:" +
+          resp.getStatus());
+    }
+    return KsmKeyInfo.getFromProtobuf(resp.getKeyInfo());
+  }
+
+  /**
+   * Deletes an existing key.
+   *
+   * @param args the args of the key.
+   * @throws IOException
+   */
+  @Override
+  public void deleteKey(KsmKeyArgs args) throws IOException {
+    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
+    KeyArgs keyArgs = KeyArgs.newBuilder()
+        .setVolumeName(args.getVolumeName())
+        .setBucketName(args.getBucketName())
+        .setKeyName(args.getKeyName()).build();
+    req.setKeyArgs(keyArgs);
+
+    final LocateKeyResponse resp;
+    try {
+      resp = rpcProxy.deleteKey(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new IOException("Delete key failed, error:" +
+          resp.getStatus());
+    }
+  }
+
+  /**
+   * Deletes an existing empty bucket from volume.
+   * @param volume - Name of the volume.
+   * @param bucket - Name of the bucket.
+   * @throws IOException
+   */
+  public void deleteBucket(String volume, String bucket) throws IOException {
+    DeleteBucketRequest.Builder req = DeleteBucketRequest.newBuilder();
+    req.setVolumeName(volume);
+    req.setBucketName(bucket);
+    final DeleteBucketResponse resp;
+    try {
+      resp = rpcProxy.deleteBucket(NULL_RPC_CONTROLLER, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    if (resp.getStatus() != Status.OK) {
+      throw new
+          IOException("Delete Bucket failed, error:" + resp.getStatus());
+    }
+  }
+
+  /**
+   * List keys in a bucket.
+   */
+  @Override
+  public List<KsmKeyInfo> listKeys(String volumeName, String bucketName,
+      String startKey, String prefix, int maxKeys) throws IOException {
+    List<KsmKeyInfo> keys = new ArrayList<>();
+    ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder();
+    reqBuilder.setVolumeName(volumeName);
+    reqBuilder.setBucketName(bucketName);
+    reqBuilder.setCount(maxKeys);
+
+    if (startKey != null) {
+      reqBuilder.setStartKey(startKey);
+    }
+
+    if (prefix != null) {
+      reqBuilder.setPrefix(prefix);
+    }
+
+    ListKeysRequest request = reqBuilder.build();
+    final ListKeysResponse resp;
+    try {
+      resp = rpcProxy.listKeys(NULL_RPC_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+    if (resp.getStatus() == Status.OK) {
+      keys.addAll(
+          resp.getKeyInfoList().stream()
+              .map(KsmKeyInfo::getFromProtobuf)
+              .collect(Collectors.toList()));
+      return keys;
+    } else {
+      throw new IOException("List Keys failed, error: "
+          + resp.getStatus());
+    }
+  }
+
+  /**
+   * Return the proxy object underlying this protocol translator.
+   *
+   * @return the proxy object underlying this protocol translator.
+   */
+  @Override
+  public Object getUnderlyingProxyObject() {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java
new file mode 100644
index 0000000..8acca8a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.ksm.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.ozone.protocol.proto
+    .KeySpaceManagerProtocolProtos.KeySpaceManagerService;
+
+/**
+ * Protocol used to communicate with KSM.
+ */
+@ProtocolInfo(protocolName =
+    "org.apache.hadoop.ozone.protocol.KeySpaceManagerProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Private
+public interface KeySpaceManagerProtocolPB
+    extends KeySpaceManagerService.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java
new file mode 100644
index 0000000..67f9f7b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java
@@ -0,0 +1,19 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.ksm.protocolPB;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/src/test/java/org/apache/hadoop/scm/TestArchive.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/src/test/java/org/apache/hadoop/scm/TestArchive.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/src/test/java/org/apache/hadoop/scm/TestArchive.java
deleted file mode 100644
index 6fff47e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/src/test/java/org/apache/hadoop/scm/TestArchive.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.scm;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.hadoop.fs.FileUtil;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.Iterator;
-import java.util.Random;
-import java.util.zip.Adler32;
-import java.util.zip.Checksum;
-
-/**
- * Test archive creation and unpacking.
- */
-public class TestArchive {
-  private static final int DIR_COUNT = 10;
-  private static final int SUB_DIR_COUNT = 3;
-  private static final int FILE_COUNT = 10;
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder();
-
-  @Rule
-  public TemporaryFolder outputFolder = new TemporaryFolder();
-
-  Checksum crc = new Adler32();
-
-  @Before
-  public void setUp() throws Exception {
-    Random r = new Random();
-    final int megaByte = 1024 * 1024;
-
-    for (int x = 0; x < DIR_COUNT; x++) {
-      File subdir = folder.newFolder(String.format("dir%d", x));
-      for (int y = 0; y < SUB_DIR_COUNT; y++) {
-        File targetDir = new File(subdir.getPath().concat(File.separator)
-            .concat(String.format("subdir%d%d", x, y)));
-        if(!targetDir.mkdirs()) {
-          throw new IOException("Failed to create subdirectory. " +
-              targetDir.toString());
-        }
-        for (int z = 0; z < FILE_COUNT; z++) {
-          Path temp = Paths.get(targetDir.getPath().concat(File.separator)
-              .concat(String.format("File%d.txt", z)));
-          byte[] buf = RandomStringUtils.randomAlphanumeric(r.nextInt(megaByte))
-              .getBytes("UTF-8");
-          Files.write(temp, buf);
-          crc.update(buf, 0, buf.length);
-        }
-      }
-    }
-  }
-
-  @Test
-  public void testArchive() throws Exception {
-    Checksum readCrc = new Adler32();
-    File archiveFile = new File(outputFolder.getRoot() + File.separator
-        + "test.container.zip");
-    long zipCheckSum = FileUtil.zip(folder.getRoot(), archiveFile);
-    Assert.assertTrue(zipCheckSum > 0);
-    File decomp = new File(outputFolder.getRoot() + File.separator +
-        "decompress");
-    if (!decomp.exists() && !decomp.mkdirs()) {
-      throw new IOException("Unable to create the destination directory. " +
-          decomp.getPath());
-    }
-
-    FileUtil.unZip(archiveFile, decomp);
-    String[] patterns = {"txt"};
-    Iterator<File> iter = FileUtils.iterateFiles(decomp, patterns, true);
-    int count = 0;
-    while (iter.hasNext()) {
-      count++;
-      byte[] buf = Files.readAllBytes(iter.next().toPath());
-      readCrc.update(buf, 0, buf.length);
-    }
-    Assert.assertEquals(DIR_COUNT * SUB_DIR_COUNT * FILE_COUNT, count);
-    Assert.assertEquals(crc.getValue(), readCrc.getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/src/test/java/org/apache/hadoop/scm/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/src/test/java/org/apache/hadoop/scm/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/src/test/java/org/apache/hadoop/scm/package-info.java
deleted file mode 100644
index 9c480d6..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/src/test/java/org/apache/hadoop/scm/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.scm;
-/**
- Test cases for SCM client classes.
- */
\ No newline at end of file


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


[8/8] hadoop git commit: HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.

Posted by xy...@apache.org.
HDFS-12259. Ozone: OzoneClient: Refactor move ozone client from hadoop-hdfs to hadoop-hdfs-client. Contributed by Nandakumar.


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

Branch: refs/heads/HDFS-7240
Commit: 43d38114e695a2dd1d2da21f848ba8142749faa3
Parents: b153dbb
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Aug 8 13:36:05 2017 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Aug 8 13:36:05 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/cblock/CBlockConfigKeys.java  | 178 +++++
 .../hadoop/ksm/helpers/KsmBucketArgs.java       | 233 ------
 .../hadoop/ksm/helpers/KsmBucketInfo.java       | 212 ------
 .../apache/hadoop/ksm/helpers/KsmKeyArgs.java   |  88 ---
 .../apache/hadoop/ksm/helpers/KsmKeyInfo.java   | 161 -----
 .../hadoop/ksm/helpers/KsmKeyLocationInfo.java  | 137 ----
 .../hadoop/ksm/helpers/KsmOzoneAclMap.java      | 110 ---
 .../hadoop/ksm/helpers/KsmVolumeArgs.java       | 223 ------
 .../apache/hadoop/ksm/helpers/VolumeArgs.java   | 140 ----
 .../apache/hadoop/ksm/helpers/package-info.java |  18 -
 .../org/apache/hadoop/ksm/package-info.java     |  21 -
 .../ksm/protocol/KeySpaceManagerProtocol.java   | 212 ------
 .../hadoop/ksm/protocol/package-info.java       |  19 -
 ...ceManagerProtocolClientSideTranslatorPB.java | 651 -----------------
 .../protocolPB/KeySpaceManagerProtocolPB.java   |  34 -
 .../hadoop/ksm/protocolPB/package-info.java     |  19 -
 .../java/org/apache/hadoop/ozone/OzoneAcl.java  |   5 +
 .../apache/hadoop/ozone/OzoneConfigKeys.java    | 119 ++++
 .../apache/hadoop/ozone/OzoneConfiguration.java |  43 ++
 .../apache/hadoop/ozone/client/OzoneBucket.java | 118 ++++
 .../apache/hadoop/ozone/client/OzoneClient.java | 415 +++++++++++
 .../hadoop/ozone/client/OzoneClientFactory.java |  95 +++
 .../hadoop/ozone/client/OzoneClientUtils.java   | 707 +++++++++++++++++++
 .../apache/hadoop/ozone/client/OzoneKey.java    | 109 +++
 .../apache/hadoop/ozone/client/OzoneVolume.java | 107 +++
 .../ozone/client/io/ChunkGroupInputStream.java  | 210 ++++++
 .../ozone/client/io/ChunkGroupOutputStream.java | 327 +++++++++
 .../ozone/client/io/LengthInputStream.java      |  49 ++
 .../client/io/OzoneContainerTranslation.java    |  51 ++
 .../ozone/client/io/OzoneInputStream.java       |  52 ++
 .../ozone/client/io/OzoneOutputStream.java      |  60 ++
 .../hadoop/ozone/client/io/package-info.java    |  23 +
 .../hadoop/ozone/client/package-info.java       |  23 +
 .../ozone/client/rest/OzoneRestClient.java      | 510 +++++++++++++
 .../ozone/client/rest/headers/Header.java       |  70 ++
 .../ozone/client/rest/headers/package-info.java |  26 +
 .../hadoop/ozone/client/rest/package-info.java  |  23 +
 .../hadoop/ozone/client/rpc/OzoneRpcClient.java | 578 +++++++++++++++
 .../hadoop/ozone/client/rpc/package-info.java   |  23 +
 .../apache/hadoop/ozone/ksm/KSMConfigKeys.java  |  77 ++
 .../hadoop/ozone/ksm/helpers/KsmBucketArgs.java | 233 ++++++
 .../hadoop/ozone/ksm/helpers/KsmBucketInfo.java | 212 ++++++
 .../hadoop/ozone/ksm/helpers/KsmKeyArgs.java    |  88 +++
 .../hadoop/ozone/ksm/helpers/KsmKeyInfo.java    | 161 +++++
 .../ozone/ksm/helpers/KsmKeyLocationInfo.java   | 137 ++++
 .../ozone/ksm/helpers/KsmOzoneAclMap.java       | 110 +++
 .../hadoop/ozone/ksm/helpers/KsmVolumeArgs.java | 223 ++++++
 .../hadoop/ozone/ksm/helpers/VolumeArgs.java    | 140 ++++
 .../hadoop/ozone/ksm/helpers/package-info.java  |  18 +
 .../apache/hadoop/ozone/ksm/package-info.java   |  21 +
 .../ksm/protocol/KeySpaceManagerProtocol.java   | 212 ++++++
 .../hadoop/ozone/ksm/protocol/package-info.java |  19 +
 ...ceManagerProtocolClientSideTranslatorPB.java | 651 +++++++++++++++++
 .../protocolPB/KeySpaceManagerProtocolPB.java   |  34 +
 .../ozone/ksm/protocolPB/package-info.java      |  19 +
 .../java/org/apache/hadoop/scm/TestArchive.java | 106 ---
 .../org/apache/hadoop/scm/package-info.java     |  21 -
 .../java/org/apache/hadoop/scm/TestArchive.java | 106 +++
 .../org/apache/hadoop/scm/package-info.java     |  21 +
 .../apache/hadoop/cblock/CBlockConfigKeys.java  | 178 -----
 .../org/apache/hadoop/cblock/CBlockManager.java |   2 +-
 .../cblock/client/CBlockVolumeClient.java       |   2 +-
 .../server/datanode/ObjectStoreHandler.java     |  12 +-
 .../hdfs/server/datanode/web/URLDispatcher.java |   2 +-
 .../org/apache/hadoop/hdfs/tools/GetConf.java   |   2 +-
 .../org/apache/hadoop/ozone/OzoneBucket.java    | 117 ---
 .../org/apache/hadoop/ozone/OzoneClient.java    | 414 -----------
 .../apache/hadoop/ozone/OzoneClientFactory.java |  69 --
 .../apache/hadoop/ozone/OzoneClientImpl.java    | 570 ---------------
 .../apache/hadoop/ozone/OzoneClientUtils.java   | 705 ------------------
 .../apache/hadoop/ozone/OzoneConfigKeys.java    | 119 ----
 .../apache/hadoop/ozone/OzoneConfiguration.java |  43 --
 .../java/org/apache/hadoop/ozone/OzoneKey.java  | 109 ---
 .../org/apache/hadoop/ozone/OzoneVolume.java    | 107 ---
 .../statemachine/DatanodeStateMachine.java      |   2 +-
 .../statemachine/EndpointStateMachine.java      |   2 +-
 .../statemachine/SCMConnectionManager.java      |   2 +-
 .../states/datanode/InitDatanodeState.java      |   2 +-
 .../hadoop/ozone/io/OzoneInputStream.java       |  53 --
 .../hadoop/ozone/io/OzoneOutputStream.java      |  62 --
 .../apache/hadoop/ozone/io/package-info.java    |  23 -
 .../apache/hadoop/ozone/ksm/BucketManager.java  |   4 +-
 .../hadoop/ozone/ksm/BucketManagerImpl.java     |   4 +-
 .../apache/hadoop/ozone/ksm/KSMConfigKeys.java  |  77 --
 .../org/apache/hadoop/ozone/ksm/KeyManager.java |   4 +-
 .../apache/hadoop/ozone/ksm/KeyManagerImpl.java |   6 +-
 .../hadoop/ozone/ksm/KeySpaceManager.java       |  16 +-
 .../hadoop/ozone/ksm/MetadataManager.java       |   6 +-
 .../hadoop/ozone/ksm/MetadataManagerImpl.java   |   6 +-
 .../apache/hadoop/ozone/ksm/VolumeManager.java  |   2 +-
 .../hadoop/ozone/ksm/VolumeManagerImpl.java     |   8 +-
 ...ceManagerProtocolServerSideTranslatorPB.java |  18 +-
 .../ozone/scm/StorageContainerManager.java      |   2 +-
 .../hadoop/ozone/scm/node/SCMNodeManager.java   |   2 +-
 .../hadoop/ozone/web/OzoneHttpServer.java       |   2 +-
 .../hadoop/ozone/web/client/OzoneBucket.java    |   4 +-
 .../ozone/web/client/OzoneRestClient.java       |   4 +-
 .../hadoop/ozone/web/client/OzoneVolume.java    |   4 +-
 .../ozone/web/handlers/BucketHandler.java       |   2 +-
 .../web/handlers/BucketProcessTemplate.java     |   2 +-
 .../hadoop/ozone/web/handlers/KeyHandler.java   |   4 +-
 .../ozone/web/handlers/KeyProcessTemplate.java  |   2 +-
 .../ozone/web/handlers/ServiceFilter.java       |   2 +-
 .../ozone/web/handlers/VolumeHandler.java       |   2 +-
 .../apache/hadoop/ozone/web/headers/Header.java |  70 --
 .../hadoop/ozone/web/headers/package-info.java  |  26 -
 .../hadoop/ozone/web/interfaces/Bucket.java     |   2 +-
 .../hadoop/ozone/web/interfaces/Keys.java       |   2 +-
 .../ozone/web/interfaces/StorageHandler.java    |   2 +-
 .../hadoop/ozone/web/interfaces/Volume.java     |   2 +-
 .../web/localstorage/LocalStorageHandler.java   |   2 +-
 .../web/localstorage/OzoneMetadataManager.java  |   2 +-
 .../LengthInputStreamMessageBodyWriter.java     |   2 +-
 .../ozone/web/ozShell/keys/ListKeyHandler.java  |   2 -
 .../hadoop/ozone/web/request/OzoneQuota.java    |   2 +-
 .../web/storage/ChunkGroupInputStream.java      | 211 ------
 .../web/storage/ChunkGroupOutputStream.java     | 325 ---------
 .../web/storage/DistributedStorageHandler.java  |  19 +-
 .../web/storage/OzoneContainerTranslation.java  | 261 -------
 .../hadoop/ozone/web/userauth/Simple.java       |   2 +-
 .../hadoop/ozone/web/utils/OzoneUtils.java      |   4 +-
 .../java/org/apache/hadoop/ozone/Corona.java    |   6 +-
 .../hadoop/ozone/TestOzoneClientImpl.java       | 357 ----------
 .../hadoop/ozone/TestOzoneClientUtils.java      | 350 ---------
 .../ozone/client/TestOzoneClientUtils.java      | 362 ++++++++++
 .../hadoop/ozone/client/package-info.java       |  23 +
 .../ozone/client/rpc/TestOzoneRpcClient.java    | 368 ++++++++++
 .../hadoop/ozone/client/rpc/package-info.java   |  23 +
 .../container/ozoneimpl/TestOzoneContainer.java |   1 -
 .../hadoop/ozone/ksm/TestBucketManagerImpl.java |   4 +-
 .../hadoop/ozone/ksm/TestChunkStreams.java      |   4 +-
 .../hadoop/ozone/web/TestOzoneHelper.java       |   2 +-
 .../hadoop/ozone/web/TestOzoneWebAccess.java    |   2 +-
 .../hadoop/ozone/web/client/TestVolume.java     |   2 +-
 134 files changed, 7278 insertions(+), 6750 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/cblock/CBlockConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/cblock/CBlockConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/cblock/CBlockConfigKeys.java
new file mode 100644
index 0000000..35b8961
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/cblock/CBlockConfigKeys.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.cblock;
+
+import static java.lang.Thread.NORM_PRIORITY;
+
+/**
+ * This class contains constants for configuration keys used in CBlock.
+ */
+public final class CBlockConfigKeys {
+  public static final String DFS_CBLOCK_SERVICERPC_ADDRESS_KEY =
+      "dfs.cblock.servicerpc-address";
+  public static final int DFS_CBLOCK_SERVICERPC_PORT_DEFAULT =
+      9810;
+  public static final String DFS_CBLOCK_SERVICERPC_HOSTNAME_DEFAULT =
+      "0.0.0.0";
+
+  public static final String DFS_CBLOCK_JSCSIRPC_ADDRESS_KEY =
+      "dfs.cblock.jscsi-address";
+
+  //The port on CBlockManager node for jSCSI to ask
+  public static final String DFS_CBLOCK_JSCSI_PORT_KEY =
+      "dfs.cblock.jscsi.port";
+  public static final int DFS_CBLOCK_JSCSI_PORT_DEFAULT =
+      9811;
+
+  public static final String DFS_CBLOCK_SERVICERPC_BIND_HOST_KEY =
+      "dfs.cblock.service.rpc-bind-host";
+  public static final String DFS_CBLOCK_JSCSIRPC_BIND_HOST_KEY =
+      "dfs.cblock.jscsi.rpc-bind-host";
+
+  // default block size is 4KB
+  public static final int DFS_CBLOCK_SERVICE_BLOCK_SIZE_DEFAULT =
+      4096;
+
+  public static final String DFS_CBLOCK_SERVICERPC_HANDLER_COUNT_KEY =
+      "dfs.storage.service.handler.count";
+  public static final int DFS_CBLOCK_SERVICERPC_HANDLER_COUNT_DEFAULT = 10;
+
+  public static final String DFS_CBLOCK_SERVICE_LEVELDB_PATH_KEY =
+      "dfs.cblock.service.leveldb.path";
+  //TODO : find a better place
+  public static final String DFS_CBLOCK_SERVICE_LEVELDB_PATH_DEFAULT =
+      "/tmp/cblock_levelDB.dat";
+
+
+  public static final String DFS_CBLOCK_DISK_CACHE_PATH_KEY =
+      "dfs.cblock.disk.cache.path";
+  public static final String DFS_CBLOCK_DISK_CACHE_PATH_DEFAULT =
+      "/tmp/cblockCacheDB";
+  /**
+   * Setting this flag to true makes the block layer compute a sha256 hash of
+   * the data and log that information along with block ID. This is very
+   * useful for doing trace based simulation of various workloads. Since it is
+   * computing a hash for each block this could be expensive, hence default
+   * is false.
+   */
+  public static final String DFS_CBLOCK_TRACE_IO = "dfs.cblock.trace.io";
+  public static final boolean DFS_CBLOCK_TRACE_IO_DEFAULT = false;
+
+  public static final String DFS_CBLOCK_ENABLE_SHORT_CIRCUIT_IO =
+      "dfs.cblock.short.circuit.io";
+  public static final boolean DFS_CBLOCK_ENABLE_SHORT_CIRCUIT_IO_DEFAULT =
+      false;
+
+  /**
+   * Cache size in 1000s of entries. 256 indicates 256 * 1024.
+   */
+  public static final String DFS_CBLOCK_CACHE_QUEUE_SIZE_KB =
+      "dfs.cblock.cache.cache.size.in.kb";
+  public static final int DFS_CBLOCK_CACHE_QUEUE_SIZE_KB_DEFAULT = 256;
+
+  /**
+   *  Minimum Number of threads that cache pool will use for background I/O.
+   */
+  public static final String DFS_CBLOCK_CACHE_CORE_MIN_POOL_SIZE =
+      "dfs.cblock.cache.core.min.pool.size";
+  public static final int DFS_CBLOCK_CACHE_CORE_MIN_POOL_SIZE_DEFAULT = 16;
+
+  /**
+   *  Maximum Number of threads that cache pool will use for background I/O.
+   */
+
+  public static final String DFS_CBLOCK_CACHE_MAX_POOL_SIZE =
+      "dfs.cblock.cache.max.pool.size";
+  public static final int DFS_CBLOCK_CACHE_MAX_POOL_SIZE_DEFAULT = 256;
+
+  /**
+   * Number of seconds to keep the Thread alive when it is idle.
+   */
+  public static final String DFS_CBLOCK_CACHE_KEEP_ALIVE_SECONDS =
+      "dfs.cblock.cache.keep.alive.seconds";
+  public static final long DFS_CBLOCK_CACHE_KEEP_ALIVE_SECONDS_DEFAULT = 60;
+
+  /**
+   * Priority of cache flusher thread, affecting the relative performance of
+   * write and read.
+   */
+  public static final String DFS_CBLOCK_CACHE_THREAD_PRIORITY =
+      "dfs.cblock.cache.thread.priority";
+  public static final int DFS_CBLOCK_CACHE_THREAD_PRIORITY_DEFAULT =
+      NORM_PRIORITY;
+
+  /**
+   * Block Buffer size in terms of blockID entries, 512 means 512 blockIDs.
+   */
+  public static final String DFS_CBLOCK_CACHE_BLOCK_BUFFER_SIZE =
+      "dfs.cblock.cache.block.buffer.size";
+  public static final int DFS_CBLOCK_CACHE_BLOCK_BUFFER_SIZE_DEFAULT = 512;
+
+  public static final String DFS_CBLOCK_BLOCK_BUFFER_FLUSH_INTERVAL_SECONDS =
+      "dfs.cblock.block.buffer.flush.interval.seconds";
+  public static final int
+      DFS_CBLOCK_BLOCK_BUFFER_FLUSH_INTERVAL_SECONDS_DEFAULT = 60;
+
+  // jscsi server settings
+  public static final String DFS_CBLOCK_JSCSI_SERVER_ADDRESS_KEY =
+      "dfs.cblock.jscsi.server.address";
+  public static final String DFS_CBLOCK_JSCSI_SERVER_ADDRESS_DEFAULT =
+      "127.0.0.1";
+  public static final String DFS_CBLOCK_JSCSI_CBLOCK_SERVER_ADDRESS_KEY =
+      "dfs.cblock.jscsi.cblock.server.address";
+  public static final String DFS_CBLOCK_JSCSI_CBLOCK_SERVER_ADDRESS_DEFAULT =
+      "127.0.0.1";
+
+  // to what address cblock server should talk to scm?
+  public static final String DFS_CBLOCK_SCM_IPADDRESS_KEY =
+      "dfs.cblock.scm.ipaddress";
+  public static final String DFS_CBLOCK_SCM_IPADDRESS_DEFAULT =
+      "127.0.0.1";
+  public static final String DFS_CBLOCK_SCM_PORT_KEY =
+      "dfs.cblock.scm.port";
+  public static final int DFS_CBLOCK_SCM_PORT_DEFAULT = 9860;
+
+  public static final String DFS_CBLOCK_CONTAINER_SIZE_GB_KEY =
+      "dfs.cblock.container.size";
+  public static final int DFS_CBLOCK_CONTAINER_SIZE_GB_DEFAULT =
+      5;
+
+  // LevelDB cache file uses an off-heap cache in LevelDB of 256 MB.
+  public static final String DFS_CBLOCK_CACHE_LEVELDB_CACHE_SIZE_MB_KEY =
+      "dfs.cblock.cache.leveldb.cache.size.mb";
+  public static final int DFS_CBLOCK_CACHE_LEVELDB_CACHE_SIZE_MB_DEFAULT = 256;
+
+  /**
+   * Cache does an best case attempt to write a block to a container.
+   * At some point of time, we will need to handle the case where we did try
+   * 64K times and is till not able to write to the container.
+   *
+   * TODO: We will need cBlock Server to allow us to do a remapping of the
+   * block location in case of failures, at that point we should reduce the
+   * retry count to a more normal number. This is approximately 18 hours of
+   * retry.
+   */
+  public static final String DFS_CBLOCK_CACHE_MAX_RETRY_KEY =
+      "dfs.cblock.cache.max.retry";
+  public static final int DFS_CBLOCK_CACHE_MAX_RETRY_DEFAULT =
+      64 * 1024;
+
+  private CBlockConfigKeys() {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java
deleted file mode 100644
index a682f5f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketArgs.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.helpers;
-
-import java.util.List;
-import java.util.stream.Collectors;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.BucketArgs;
-import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
-
-/**
- * A class that encapsulates Bucket Arguments.
- */
-public final class KsmBucketArgs {
-  /**
-   * Name of the volume in which the bucket belongs to.
-   */
-  private final String volumeName;
-  /**
-   * Name of the bucket.
-   */
-  private final String bucketName;
-  /**
-   * ACL's that are to be added for the bucket.
-   */
-  private List<OzoneAcl> addAcls;
-  /**
-   * ACL's that are to be removed from the bucket.
-   */
-  private List<OzoneAcl> removeAcls;
-  /**
-   * Bucket Version flag.
-   */
-  private Boolean isVersionEnabled;
-  /**
-   * Type of storage to be used for this bucket.
-   * [RAM_DISK, SSD, DISK, ARCHIVE]
-   */
-  private StorageType storageType;
-
-  /**
-   * Private constructor, constructed via builder.
-   * @param volumeName - Volume name.
-   * @param bucketName - Bucket name.
-   * @param addAcls - ACL's to be added.
-   * @param removeAcls - ACL's to be removed.
-   * @param isVersionEnabled - Bucket version flag.
-   * @param storageType - Storage type to be used.
-   */
-  private KsmBucketArgs(String volumeName, String bucketName,
-      List<OzoneAcl> addAcls, List<OzoneAcl> removeAcls,
-      Boolean isVersionEnabled, StorageType storageType) {
-    this.volumeName = volumeName;
-    this.bucketName = bucketName;
-    this.addAcls = addAcls;
-    this.removeAcls = removeAcls;
-    this.isVersionEnabled = isVersionEnabled;
-    this.storageType = storageType;
-  }
-
-  /**
-   * Returns the Volume Name.
-   * @return String.
-   */
-  public String getVolumeName() {
-    return volumeName;
-  }
-
-  /**
-   * Returns the Bucket Name.
-   * @return String
-   */
-  public String getBucketName() {
-    return bucketName;
-  }
-
-  /**
-   * Returns the ACL's that are to be added.
-   * @return List<OzoneAclInfo>
-   */
-  public List<OzoneAcl> getAddAcls() {
-    return addAcls;
-  }
-
-  /**
-   * Returns the ACL's that are to be removed.
-   * @return List<OzoneAclInfo>
-   */
-  public List<OzoneAcl> getRemoveAcls() {
-    return removeAcls;
-  }
-
-  /**
-   * Returns true if bucket version is enabled, else false.
-   * @return isVersionEnabled
-   */
-  public Boolean getIsVersionEnabled() {
-    return isVersionEnabled;
-  }
-
-  /**
-   * Returns the type of storage to be used.
-   * @return StorageType
-   */
-  public StorageType getStorageType() {
-    return storageType;
-  }
-
-  /**
-   * Returns new builder class that builds a KsmBucketArgs.
-   *
-   * @return Builder
-   */
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * Builder for KsmBucketArgs.
-   */
-  public static class Builder {
-    private String volumeName;
-    private String bucketName;
-    private List<OzoneAcl> addAcls;
-    private List<OzoneAcl> removeAcls;
-    private Boolean isVersionEnabled;
-    private StorageType storageType;
-
-    public Builder setVolumeName(String volume) {
-      this.volumeName = volume;
-      return this;
-    }
-
-    public Builder setBucketName(String bucket) {
-      this.bucketName = bucket;
-      return this;
-    }
-
-    public Builder setAddAcls(List<OzoneAcl> acls) {
-      this.addAcls = acls;
-      return this;
-    }
-
-    public Builder setRemoveAcls(List<OzoneAcl> acls) {
-      this.removeAcls = acls;
-      return this;
-    }
-
-    public Builder setIsVersionEnabled(Boolean versionFlag) {
-      this.isVersionEnabled = versionFlag;
-      return this;
-    }
-
-    public Builder setStorageType(StorageType storage) {
-      this.storageType = storage;
-      return this;
-    }
-
-    /**
-     * Constructs the KsmBucketArgs.
-     * @return instance of KsmBucketArgs.
-     */
-    public KsmBucketArgs build() {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
-      return new KsmBucketArgs(volumeName, bucketName, addAcls,
-          removeAcls, isVersionEnabled, storageType);
-    }
-  }
-
-  /**
-   * Creates BucketArgs protobuf from KsmBucketArgs.
-   */
-  public BucketArgs getProtobuf() {
-    BucketArgs.Builder builder = BucketArgs.newBuilder();
-    builder.setVolumeName(volumeName)
-        .setBucketName(bucketName);
-    if(addAcls != null && !addAcls.isEmpty()) {
-      builder.addAllAddAcls(addAcls.stream().map(
-          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
-    }
-    if(removeAcls != null && !removeAcls.isEmpty()) {
-      builder.addAllRemoveAcls(removeAcls.stream().map(
-          KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
-    }
-    if(isVersionEnabled != null) {
-      builder.setIsVersionEnabled(isVersionEnabled);
-    }
-    if(storageType != null) {
-      builder.setStorageType(
-          PBHelperClient.convertStorageType(storageType));
-    }
-    return builder.build();
-  }
-
-  /**
-   * Parses BucketInfo protobuf and creates KsmBucketArgs.
-   * @param bucketArgs
-   * @return instance of KsmBucketArgs
-   */
-  public static KsmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
-    return new KsmBucketArgs(bucketArgs.getVolumeName(),
-        bucketArgs.getBucketName(),
-        bucketArgs.getAddAclsList().stream().map(
-            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
-        bucketArgs.getRemoveAclsList().stream().map(
-            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
-        bucketArgs.hasIsVersionEnabled() ?
-            bucketArgs.getIsVersionEnabled() : null,
-        bucketArgs.hasStorageType() ? PBHelperClient.convertStorageType(
-            bucketArgs.getStorageType()) : null);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java
deleted file mode 100644
index effbb11..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmBucketInfo.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.helpers;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.BucketInfo;
-import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.stream.Collectors;
-
-/**
- * A class that encapsulates Bucket Info.
- */
-public final class KsmBucketInfo {
-  /**
-   * Name of the volume in which the bucket belongs to.
-   */
-  private final String volumeName;
-  /**
-   * Name of the bucket.
-   */
-  private final String bucketName;
-  /**
-   * ACL Information.
-   */
-  private List<OzoneAcl> acls;
-  /**
-   * Bucket Version flag.
-   */
-  private Boolean isVersionEnabled;
-  /**
-   * Type of storage to be used for this bucket.
-   * [RAM_DISK, SSD, DISK, ARCHIVE]
-   */
-  private StorageType storageType;
-
-  /**
-   * Private constructor, constructed via builder.
-   * @param volumeName - Volume name.
-   * @param bucketName - Bucket name.
-   * @param acls - list of ACLs.
-   * @param isVersionEnabled - Bucket version flag.
-   * @param storageType - Storage type to be used.
-   */
-  private KsmBucketInfo(String volumeName, String bucketName,
-                        List<OzoneAcl> acls, boolean isVersionEnabled,
-                        StorageType storageType) {
-    this.volumeName = volumeName;
-    this.bucketName = bucketName;
-    this.acls = acls;
-    this.isVersionEnabled = isVersionEnabled;
-    this.storageType = storageType;
-  }
-
-  /**
-   * Returns the Volume Name.
-   * @return String.
-   */
-  public String getVolumeName() {
-    return volumeName;
-  }
-
-  /**
-   * Returns the Bucket Name.
-   * @return String
-   */
-  public String getBucketName() {
-    return bucketName;
-  }
-
-  /**
-   * Returns the ACL's associated with this bucket.
-   * @return List<OzoneAcl>
-   */
-  public List<OzoneAcl> getAcls() {
-    return acls;
-  }
-
-  /**
-   * Returns true if bucket version is enabled, else false.
-   * @return isVersionEnabled
-   */
-  public boolean getIsVersionEnabled() {
-    return isVersionEnabled;
-  }
-
-  /**
-   * Returns the type of storage to be used.
-   * @return StorageType
-   */
-  public StorageType getStorageType() {
-    return storageType;
-  }
-
-  /**
-   * Returns new builder class that builds a KsmBucketInfo.
-   *
-   * @return Builder
-   */
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * Builder for KsmBucketInfo.
-   */
-  public static class Builder {
-    private String volumeName;
-    private String bucketName;
-    private List<OzoneAcl> acls;
-    private Boolean isVersionEnabled;
-    private StorageType storageType;
-
-    Builder() {
-      //Default values
-      this.acls = new LinkedList<>();
-      this.isVersionEnabled = false;
-      this.storageType = StorageType.DISK;
-    }
-
-    public Builder setVolumeName(String volume) {
-      this.volumeName = volume;
-      return this;
-    }
-
-    public Builder setBucketName(String bucket) {
-      this.bucketName = bucket;
-      return this;
-    }
-
-    public Builder setAcls(List<OzoneAcl> listOfAcls) {
-      this.acls = listOfAcls;
-      return this;
-    }
-
-    public Builder setIsVersionEnabled(Boolean versionFlag) {
-      this.isVersionEnabled = versionFlag;
-      return this;
-    }
-
-    public Builder setStorageType(StorageType storage) {
-      this.storageType = storage;
-      return this;
-    }
-
-    /**
-     * Constructs the KsmBucketInfo.
-     * @return instance of KsmBucketInfo.
-     */
-    public KsmBucketInfo build() {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
-      Preconditions.checkNotNull(acls);
-      Preconditions.checkNotNull(isVersionEnabled);
-      Preconditions.checkNotNull(storageType);
-      return new KsmBucketInfo(volumeName, bucketName, acls,
-          isVersionEnabled, storageType);
-    }
-  }
-
-  /**
-   * Creates BucketInfo protobuf from KsmBucketInfo.
-   */
-  public BucketInfo getProtobuf() {
-    return BucketInfo.newBuilder()
-        .setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .addAllAcls(acls.stream().map(
-            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()))
-        .setIsVersionEnabled(isVersionEnabled)
-        .setStorageType(PBHelperClient.convertStorageType(
-            storageType))
-        .build();
-  }
-
-  /**
-   * Parses BucketInfo protobuf and creates KsmBucketInfo.
-   * @param bucketInfo
-   * @return instance of KsmBucketInfo
-   */
-  public static KsmBucketInfo getFromProtobuf(BucketInfo bucketInfo) {
-    return new KsmBucketInfo(
-        bucketInfo.getVolumeName(),
-        bucketInfo.getBucketName(),
-        bucketInfo.getAclsList().stream().map(
-            KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
-        bucketInfo.getIsVersionEnabled(),
-        PBHelperClient.convertStorageType(
-            bucketInfo.getStorageType()));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyArgs.java
deleted file mode 100644
index a034ed3..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyArgs.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.helpers;
-
-/**
- * Args for key. Client use this to specify key's attributes on  key creation
- * (putKey()).
- */
-public final class KsmKeyArgs {
-  private final String volumeName;
-  private final String bucketName;
-  private final String keyName;
-
-  private final long dataSize;
-
-  private KsmKeyArgs(String volumeName, String bucketName, String keyName,
-      long dataSize) {
-    this.volumeName = volumeName;
-    this.bucketName = bucketName;
-    this.keyName = keyName;
-    this.dataSize = dataSize;
-  }
-
-  public String getVolumeName() {
-    return volumeName;
-  }
-
-  public String getBucketName() {
-    return bucketName;
-  }
-
-  public String getKeyName() {
-    return keyName;
-  }
-
-  public long getDataSize() {
-    return dataSize;
-  }
-
-  /**
-   * Builder class of KsmKeyArgs.
-   */
-  public static class Builder {
-    private String volumeName;
-    private String bucketName;
-    private String keyName;
-    private long dataSize;
-
-    public Builder setVolumeName(String volume) {
-      this.volumeName = volume;
-      return this;
-    }
-
-    public Builder setBucketName(String bucket) {
-      this.bucketName = bucket;
-      return this;
-    }
-
-    public Builder setKeyName(String key) {
-      this.keyName = key;
-      return this;
-    }
-
-    public Builder setDataSize(long size) {
-      this.dataSize = size;
-      return this;
-    }
-
-    public KsmKeyArgs build() {
-      return new KsmKeyArgs(volumeName, bucketName, keyName, dataSize);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyInfo.java
deleted file mode 100644
index f46ec89..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyInfo.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.helpers;
-
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyInfo;
-
-import java.util.List;
-import java.util.stream.Collectors;
-
-/**
- * Args for key block. The block instance for the key requested in putKey.
- * This is returned from KSM to client, and client use class to talk to
- * datanode. Also, this is the metadata written to ksm.db on server side.
- */
-public final class KsmKeyInfo {
-  private final String volumeName;
-  private final String bucketName;
-  // name of key client specified
-  private final String keyName;
-  private final long dataSize;
-  private List<KsmKeyLocationInfo> keyLocationList;
-  private final long creationTime;
-  private final long modificationTime;
-
-  private KsmKeyInfo(String volumeName, String bucketName, String keyName,
-      List<KsmKeyLocationInfo> locationInfos, long dataSize, long creationTime,
-      long modificationTime) {
-    this.volumeName = volumeName;
-    this.bucketName = bucketName;
-    this.keyName = keyName;
-    this.dataSize = dataSize;
-    this.keyLocationList = locationInfos;
-    this.creationTime = creationTime;
-    this.modificationTime = modificationTime;
-  }
-
-  public String getVolumeName() {
-    return volumeName;
-  }
-
-  public String getBucketName() {
-    return bucketName;
-  }
-
-  public String getKeyName() {
-    return keyName;
-  }
-
-  public long getDataSize() {
-    return dataSize;
-  }
-
-  public List<KsmKeyLocationInfo> getKeyLocationList() {
-    return keyLocationList;
-  }
-
-  public long getCreationTime() {
-    return creationTime;
-  }
-
-  public long getModificationTime() {
-    return modificationTime;
-  }
-
-  /**
-   * Builder of KsmKeyInfo.
-   */
-  public static class Builder {
-    private String volumeName;
-    private String bucketName;
-    private String keyName;
-    private long dataSize;
-    private List<KsmKeyLocationInfo> ksmKeyLocationInfos;
-    private long creationTime;
-    private long modificationTime;
-
-    public Builder setVolumeName(String volume) {
-      this.volumeName = volume;
-      return this;
-    }
-
-    public Builder setBucketName(String bucket) {
-      this.bucketName = bucket;
-      return this;
-    }
-
-    public Builder setKeyName(String key) {
-      this.keyName = key;
-      return this;
-    }
-
-    public Builder setKsmKeyLocationInfos(
-        List<KsmKeyLocationInfo> ksmKeyLocationInfoList) {
-      this.ksmKeyLocationInfos = ksmKeyLocationInfoList;
-      return this;
-    }
-
-    public Builder setDataSize(long size) {
-      this.dataSize = size;
-      return this;
-    }
-
-    public Builder setCreationTime(long creationTime) {
-      this.creationTime = creationTime;
-      return this;
-    }
-
-    public Builder setModificationTime(long modificationTime) {
-      this.modificationTime = modificationTime;
-      return this;
-    }
-
-    public KsmKeyInfo build() {
-      return new KsmKeyInfo(
-          volumeName, bucketName, keyName, ksmKeyLocationInfos,
-          dataSize, creationTime, modificationTime);
-    }
-  }
-
-  public KeyInfo getProtobuf() {
-    return KeyInfo.newBuilder()
-        .setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setKeyName(keyName)
-        .setDataSize(dataSize)
-        .addAllKeyLocationList(keyLocationList.stream()
-            .map(KsmKeyLocationInfo::getProtobuf).collect(Collectors.toList()))
-        .setCreationTime(creationTime)
-        .setModificationTime(modificationTime)
-        .build();
-  }
-
-  public static KsmKeyInfo getFromProtobuf(KeyInfo keyInfo) {
-    return new KsmKeyInfo(
-        keyInfo.getVolumeName(),
-        keyInfo.getBucketName(),
-        keyInfo.getKeyName(),
-        keyInfo.getKeyLocationListList().stream()
-            .map(KsmKeyLocationInfo::getFromProtobuf)
-            .collect(Collectors.toList()),
-        keyInfo.getDataSize(),
-        keyInfo.getCreationTime(),
-        keyInfo.getModificationTime());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyLocationInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyLocationInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyLocationInfo.java
deleted file mode 100644
index 62d20f6..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmKeyLocationInfo.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.ksm.helpers;
-
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyLocation;
-
-/**
- * One key can be too huge to fit in one container. In which case it gets split
- * into a number of subkeys. This class represents one such subkey instance.
- */
-public final class KsmKeyLocationInfo {
-  private final String containerName;
-  // name of the block id SCM assigned for the key
-  private final String blockID;
-  private final boolean shouldCreateContainer;
-  // the id of this subkey in all the subkeys.
-  private final int index;
-  private final long length;
-  private final long offset;
-
-  private KsmKeyLocationInfo(String containerName,
-      String blockID, boolean shouldCreateContainer, int index,
-      long length, long offset) {
-    this.containerName = containerName;
-    this.blockID = blockID;
-    this.shouldCreateContainer = shouldCreateContainer;
-    this.index = index;
-    this.length = length;
-    this.offset = offset;
-  }
-
-  public String getContainerName() {
-    return containerName;
-  }
-
-  public String getBlockID() {
-    return blockID;
-  }
-
-  public boolean getShouldCreateContainer() {
-    return shouldCreateContainer;
-  }
-
-  public int getIndex() {
-    return index;
-  }
-
-  public long getLength() {
-    return length;
-  }
-
-  public long getOffset() {
-    return offset;
-  }
-
-  /**
-   * Builder of KsmKeyLocationInfo.
-   */
-  public static class Builder {
-    private String containerName;
-    private String blockID;
-    private boolean shouldCreateContainer;
-    // the id of this subkey in all the subkeys.
-    private int index;
-    private long length;
-    private long offset;
-    public Builder setContainerName(String container) {
-      this.containerName = container;
-      return this;
-    }
-
-    public Builder setBlockID(String block) {
-      this.blockID = block;
-      return this;
-    }
-
-    public Builder setShouldCreateContainer(boolean create) {
-      this.shouldCreateContainer = create;
-      return this;
-    }
-
-    public Builder setIndex(int id) {
-      this.index = id;
-      return this;
-    }
-
-    public Builder setLength(long len) {
-      this.length = len;
-      return this;
-    }
-
-    public Builder setOffset(long off) {
-      this.offset = off;
-      return this;
-    }
-
-    public KsmKeyLocationInfo build() {
-      return new KsmKeyLocationInfo(containerName, blockID,
-          shouldCreateContainer, index, length, offset);
-    }
-  }
-
-  public KeyLocation getProtobuf() {
-    return KeyLocation.newBuilder()
-        .setContainerName(containerName)
-        .setBlockID(blockID)
-        .setShouldCreateContainer(shouldCreateContainer)
-        .setIndex(index)
-        .setLength(length)
-        .setOffset(offset)
-        .build();
-  }
-
-  public static KsmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) {
-    return new KsmKeyLocationInfo(
-        keyLocation.getContainerName(),
-        keyLocation.getBlockID(),
-        keyLocation.getShouldCreateContainer(),
-        keyLocation.getIndex(),
-        keyLocation.getLength(),
-        keyLocation.getOffset());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmOzoneAclMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmOzoneAclMap.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmOzoneAclMap.java
deleted file mode 100644
index 1c4c9cb..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmOzoneAclMap.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ksm.helpers;
-
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
-
-import java.util.List;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.ArrayList;
-import java.util.HashMap;
-
-/**
- * This helper class keeps a map of all user and their permissions.
- */
-public class KsmOzoneAclMap {
-  // per Acl Type user:rights map
-  private ArrayList<Map<String, OzoneAclRights>> aclMaps;
-
-  KsmOzoneAclMap() {
-    aclMaps = new ArrayList<>();
-    for (OzoneAclType aclType : OzoneAclType.values()) {
-      aclMaps.add(aclType.ordinal(), new HashMap<>());
-    }
-  }
-
-  private Map<String, OzoneAclRights> getMap(OzoneAclType type) {
-    return aclMaps.get(type.ordinal());
-  }
-
-  // For a given acl type and user, get the stored acl
-  private OzoneAclRights getAcl(OzoneAclType type, String user) {
-    return getMap(type).get(user);
-  }
-
-  // Add a new acl to the map
-  public void addAcl(OzoneAclInfo acl) {
-    getMap(acl.getType()).put(acl.getName(), acl.getRights());
-  }
-
-  // for a given acl, check if the user has access rights
-  public boolean hasAccess(OzoneAclInfo acl) {
-    OzoneAclRights storedRights = getAcl(acl.getType(), acl.getName());
-    if (storedRights != null) {
-      switch (acl.getRights()) {
-      case READ:
-        return (storedRights == OzoneAclRights.READ)
-            || (storedRights == OzoneAclRights.READ_WRITE);
-      case WRITE:
-        return (storedRights == OzoneAclRights.WRITE)
-            || (storedRights == OzoneAclRights.READ_WRITE);
-      case READ_WRITE:
-        return (storedRights == OzoneAclRights.READ_WRITE);
-      default:
-        return false;
-      }
-    } else {
-      return false;
-    }
-  }
-
-  // Convert this map to OzoneAclInfo Protobuf List
-  public List<OzoneAclInfo> ozoneAclGetProtobuf() {
-    List<OzoneAclInfo> aclList = new LinkedList<>();
-    for (OzoneAclType type: OzoneAclType.values()) {
-      for (Map.Entry<String, OzoneAclRights> entry :
-          aclMaps.get(type.ordinal()).entrySet()) {
-        OzoneAclInfo aclInfo = OzoneAclInfo.newBuilder()
-            .setName(entry.getKey())
-            .setType(type)
-            .setRights(entry.getValue())
-            .build();
-        aclList.add(aclInfo);
-      }
-    }
-
-    return aclList;
-  }
-
-  // Create map from list of OzoneAclInfos
-  public static KsmOzoneAclMap ozoneAclGetFromProtobuf(
-      List<OzoneAclInfo> aclList) {
-    KsmOzoneAclMap aclMap = new KsmOzoneAclMap();
-    for (OzoneAclInfo acl : aclList) {
-      aclMap.addAcl(acl);
-    }
-    return aclMap;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmVolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmVolumeArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmVolumeArgs.java
deleted file mode 100644
index f7c6593..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/KsmVolumeArgs.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.helpers;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.VolumeInfo;
-import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.KeyValue;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-
-/**
- * A class that encapsulates the KsmVolumeArgs Args.
- */
-public final class KsmVolumeArgs {
-  private final String adminName;
-  private final String ownerName;
-  private final String volume;
-  private final long creationTime;
-  private final long quotaInBytes;
-  private final Map<String, String> keyValueMap;
-  private final KsmOzoneAclMap aclMap;
-
-  /**
-   * Private constructor, constructed via builder.
-   * @param adminName  - Administrator's name.
-   * @param ownerName  - Volume owner's name
-   * @param volume - volume name
-   * @param quotaInBytes - Volume Quota in bytes.
-   * @param keyValueMap - keyValue map.
-   * @param aclMap - User to access rights map.
-   * @param creationTime - Volume creation time.
-   */
-  private KsmVolumeArgs(String adminName, String ownerName, String volume,
-                        long quotaInBytes, Map<String, String> keyValueMap,
-                        KsmOzoneAclMap aclMap, long creationTime) {
-    this.adminName = adminName;
-    this.ownerName = ownerName;
-    this.volume = volume;
-    this.quotaInBytes = quotaInBytes;
-    this.keyValueMap = keyValueMap;
-    this.aclMap = aclMap;
-    this.creationTime = creationTime;
-  }
-
-  /**
-   * Returns the Admin Name.
-   * @return String.
-   */
-  public String getAdminName() {
-    return adminName;
-  }
-
-  /**
-   * Returns the owner Name.
-   * @return String
-   */
-  public String getOwnerName() {
-    return ownerName;
-  }
-
-  /**
-   * Returns the volume Name.
-   * @return String
-   */
-  public String getVolume() {
-    return volume;
-  }
-
-  /**
-   * Returns creation time.
-   * @return long
-   */
-  public long getCreationTime() {
-    return creationTime;
-  }
-
-  /**
-   * Returns Quota in Bytes.
-   * @return long, Quota in bytes.
-   */
-  public long getQuotaInBytes() {
-    return quotaInBytes;
-  }
-
-  public Map<String, String> getKeyValueMap() {
-    return keyValueMap;
-  }
-
-  public KsmOzoneAclMap getAclMap() {
-    return aclMap;
-  }
-  /**
-   * Returns new builder class that builds a KsmVolumeArgs.
-   *
-   * @return Builder
-   */
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * Builder for KsmVolumeArgs.
-   */
-  public static class Builder {
-    private String adminName;
-    private String ownerName;
-    private String volume;
-    private long creationTime;
-    private long quotaInBytes;
-    private Map<String, String> keyValueMap;
-    private KsmOzoneAclMap aclMap;
-
-    /**
-     * Constructs a builder.
-     */
-    Builder() {
-      keyValueMap = new HashMap<>();
-      aclMap = new KsmOzoneAclMap();
-    }
-
-    public Builder setAdminName(String adminName) {
-      this.adminName = adminName;
-      return this;
-    }
-
-    public Builder setOwnerName(String ownerName) {
-      this.ownerName = ownerName;
-      return this;
-    }
-
-    public Builder setVolume(String volume) {
-      this.volume = volume;
-      return this;
-    }
-
-    public Builder setCreationTime(long createdOn) {
-      this.creationTime = createdOn;
-      return this;
-    }
-
-    public Builder setQuotaInBytes(long quotaInBytes) {
-      this.quotaInBytes = quotaInBytes;
-      return this;
-    }
-
-    public Builder addMetadata(String key, String value) {
-      keyValueMap.put(key, value); // overwrite if present.
-      return this;
-    }
-
-    public Builder addOzoneAcls(OzoneAclInfo acl) throws IOException {
-      aclMap.addAcl(acl);
-      return this;
-    }
-
-    /**
-     * Constructs a CreateVolumeArgument.
-     * @return CreateVolumeArgs.
-     */
-    public KsmVolumeArgs build() {
-      Preconditions.checkNotNull(adminName);
-      Preconditions.checkNotNull(ownerName);
-      Preconditions.checkNotNull(volume);
-      return new KsmVolumeArgs(adminName, ownerName, volume, quotaInBytes,
-          keyValueMap, aclMap, creationTime);
-    }
-  }
-
-  public VolumeInfo getProtobuf() {
-    List<KeyValue> metadataList = new LinkedList<>();
-    for (Map.Entry<String, String> entry : keyValueMap.entrySet()) {
-      metadataList.add(KeyValue.newBuilder().setKey(entry.getKey()).
-          setValue(entry.getValue()).build());
-    }
-    List<OzoneAclInfo> aclList = aclMap.ozoneAclGetProtobuf();
-
-    return VolumeInfo.newBuilder()
-        .setAdminName(adminName)
-        .setOwnerName(ownerName)
-        .setVolume(volume)
-        .setQuotaInBytes(quotaInBytes)
-        .addAllMetadata(metadataList)
-        .addAllVolumeAcls(aclList)
-        .setCreationTime(creationTime)
-        .build();
-  }
-
-  public static KsmVolumeArgs getFromProtobuf(VolumeInfo volInfo) {
-    Map<String, String> kvMap = volInfo.getMetadataList().stream()
-        .collect(Collectors.toMap(KeyValue::getKey,
-            KeyValue::getValue));
-    KsmOzoneAclMap aclMap =
-        KsmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList());
-
-    return new KsmVolumeArgs(volInfo.getAdminName(), volInfo.getOwnerName(),
-        volInfo.getVolume(), volInfo.getQuotaInBytes(), kvMap, aclMap,
-        volInfo.getCreationTime());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/VolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/VolumeArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/VolumeArgs.java
deleted file mode 100644
index 472af65..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/VolumeArgs.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.helpers;
-
-import com.google.common.base.Preconditions;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * A class that encapsulates the createVolume Args.
- */
-public final class VolumeArgs {
-  private final String adminName;
-  private final String ownerName;
-  private final String volume;
-  private final long quotaInBytes;
-  private final Map<String, String> extendedAttributes;
-
-  /**
-   * Private constructor, constructed via builder.
-   *
-   * @param adminName - Administrator name.
-   * @param ownerName - Volume owner's name
-   * @param volume - volume name
-   * @param quotaInBytes - Volume Quota in bytes.
-   * @param keyValueMap - keyValue map.
-   */
-  private VolumeArgs(String adminName, String ownerName, String volume,
-      long quotaInBytes, Map<String, String> keyValueMap) {
-    this.adminName = adminName;
-    this.ownerName = ownerName;
-    this.volume = volume;
-    this.quotaInBytes = quotaInBytes;
-    this.extendedAttributes = keyValueMap;
-  }
-
-  /**
-   * Returns the Admin Name.
-   *
-   * @return String.
-   */
-  public String getAdminName() {
-    return adminName;
-  }
-
-  /**
-   * Returns the owner Name.
-   *
-   * @return String
-   */
-  public String getOwnerName() {
-    return ownerName;
-  }
-
-  /**
-   * Returns the volume Name.
-   *
-   * @return String
-   */
-  public String getVolume() {
-    return volume;
-  }
-
-  /**
-   * Returns Quota in Bytes.
-   *
-   * @return long, Quota in bytes.
-   */
-  public long getQuotaInBytes() {
-    return quotaInBytes;
-  }
-
-  public Map<String, String> getExtendedAttributes() {
-    return extendedAttributes;
-  }
-
-  static class Builder {
-    private String adminName;
-    private String ownerName;
-    private String volume;
-    private long quotaInBytes;
-    private Map<String, String> extendedAttributes;
-
-    /**
-     * Constructs a builder.
-     */
-    Builder() {
-      extendedAttributes = new HashMap<>();
-    }
-
-    public void setAdminName(String adminName) {
-      this.adminName = adminName;
-    }
-
-    public void setOwnerName(String ownerName) {
-      this.ownerName = ownerName;
-    }
-
-    public void setVolume(String volume) {
-      this.volume = volume;
-    }
-
-    public void setQuotaInBytes(long quotaInBytes) {
-      this.quotaInBytes = quotaInBytes;
-    }
-
-    public void addMetadata(String key, String value) {
-      extendedAttributes.put(key, value); // overwrite if present.
-    }
-
-    /**
-     * Constructs a CreateVolumeArgument.
-     *
-     * @return CreateVolumeArgs.
-     */
-    public VolumeArgs build() {
-      Preconditions.checkNotNull(adminName);
-      Preconditions.checkNotNull(ownerName);
-      Preconditions.checkNotNull(volume);
-      return new VolumeArgs(adminName, ownerName, volume, quotaInBytes,
-          extendedAttributes);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/package-info.java
deleted file mode 100644
index 8e48773..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/helpers/package-info.java
+++ /dev/null
@@ -1,18 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.helpers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/package-info.java
deleted file mode 100644
index 5b208fa..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm;
-/**
- This package contains client side protocol library to communicate with KSM.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocol/KeySpaceManagerProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocol/KeySpaceManagerProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocol/KeySpaceManagerProtocol.java
deleted file mode 100644
index 2ff8e36..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocol/KeySpaceManagerProtocol.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ksm.protocol;
-
-import org.apache.hadoop.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ksm.helpers.KsmBucketInfo;
-import org.apache.hadoop.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ksm.helpers.KsmVolumeArgs;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import java.io.IOException;
-import java.util.List;
-
-/**
- * Protocol to talk to KSM.
- */
-public interface KeySpaceManagerProtocol {
-
-  /**
-   * Creates a volume.
-   * @param args - Arguments to create Volume.
-   * @throws IOException
-   */
-  void createVolume(KsmVolumeArgs args) throws IOException;
-
-  /**
-   * Changes the owner of a volume.
-   * @param volume  - Name of the volume.
-   * @param owner - Name of the owner.
-   * @throws IOException
-   */
-  void setOwner(String volume, String owner) throws IOException;
-
-  /**
-   * Changes the Quota on a volume.
-   * @param volume - Name of the volume.
-   * @param quota - Quota in bytes.
-   * @throws IOException
-   */
-  void setQuota(String volume, long quota) throws IOException;
-
-  /**
-   * Checks if the specified user can access this volume.
-   * @param volume - volume
-   * @param userAcl - user acls which needs to be checked for access
-   * @return true if the user has required access for the volume,
-   *         false otherwise
-   * @throws IOException
-   */
-  boolean checkVolumeAccess(String volume, OzoneAclInfo userAcl)
-      throws IOException;
-
-  /**
-   * Gets the volume information.
-   * @param volume - Volume name.
-   * @return VolumeArgs or exception is thrown.
-   * @throws IOException
-   */
-  KsmVolumeArgs getVolumeInfo(String volume) throws IOException;
-
-  /**
-   * Deletes an existing empty volume.
-   * @param volume - Name of the volume.
-   * @throws IOException
-   */
-  void deleteVolume(String volume) throws IOException;
-
-  /**
-   * Lists volume owned by a specific user.
-   * @param userName - user name
-   * @param prefix  - Filter prefix -- Return only entries that match this.
-   * @param prevKey - Previous key -- List starts from the next from the prevkey
-   * @param maxKeys - Max number of keys to return.
-   * @return List of Volumes.
-   * @throws IOException
-   */
-  List<KsmVolumeArgs> listVolumeByUser(String userName, String prefix, String
-      prevKey, int maxKeys) throws IOException;
-
-  /**
-   * Lists volume all volumes in the cluster.
-   * @param prefix  - Filter prefix -- Return only entries that match this.
-   * @param prevKey - Previous key -- List starts from the next from the prevkey
-   * @param maxKeys - Max number of keys to return.
-   * @return List of Volumes.
-   * @throws IOException
-   */
-  List<KsmVolumeArgs> listAllVolumes(String prefix, String
-      prevKey, int maxKeys) throws IOException;
-
-  /**
-   * Creates a bucket.
-   * @param bucketInfo - BucketInfo to create Bucket.
-   * @throws IOException
-   */
-  void createBucket(KsmBucketInfo bucketInfo) throws IOException;
-
-  /**
-   * Gets the bucket information.
-   * @param volumeName - Volume name.
-   * @param bucketName - Bucket name.
-   * @return KsmBucketInfo or exception is thrown.
-   * @throws IOException
-   */
-  KsmBucketInfo getBucketInfo(String volumeName, String bucketName)
-      throws IOException;
-
-  /**
-   * Sets bucket property from args.
-   * @param args - BucketArgs.
-   * @throws IOException
-   */
-  void setBucketProperty(KsmBucketArgs args) throws IOException;
-
-  /**
-   * Allocate a block to a container, the block is returned to the client.
-   *
-   * @param args the args of the key.
-   * @return KsmKeyInfo isntacne that client uses to talk to container.
-   * @throws IOException
-   */
-  KsmKeyInfo allocateKey(KsmKeyArgs args) throws IOException;
-
-  /**
-   * Look up for the container of an existing key.
-   *
-   * @param args the args of the key.
-   * @return KsmKeyInfo isntacne that client uses to talk to container.
-   * @throws IOException
-   */
-  KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException;
-
-  /**
-   * Deletes an existing key.
-   *
-   * @param args the args of the key.
-   * @throws IOException
-   */
-  void deleteKey(KsmKeyArgs args) throws IOException;
-
-  /**
-   * Deletes an existing empty bucket from volume.
-   * @param volume - Name of the volume.
-   * @param bucket - Name of the bucket.
-   * @throws IOException
-   */
-  void deleteBucket(String volume, String bucket) throws IOException;
-
-  /**
-   * Returns a list of buckets represented by {@link KsmBucketInfo}
-   * in the given volume. Argument volumeName is required, others
-   * are optional.
-   *
-   * @param volumeName
-   *   the name of the volume.
-   * @param startBucketName
-   *   the start bucket name, only the buckets whose name is
-   *   after this value will be included in the result.
-   * @param bucketPrefix
-   *   bucket name prefix, only the buckets whose name has
-   *   this prefix will be included in the result.
-   * @param maxNumOfBuckets
-   *   the maximum number of buckets to return. It ensures
-   *   the size of the result will not exceed this limit.
-   * @return a list of buckets.
-   * @throws IOException
-   */
-  List<KsmBucketInfo> listBuckets(String volumeName,
-      String startBucketName, String bucketPrefix, int maxNumOfBuckets)
-      throws IOException;
-
-  /**
-   * Returns a list of keys represented by {@link KsmKeyInfo}
-   * in the given bucket. Argument volumeName, bucketName is required,
-   * others are optional.
-   *
-   * @param volumeName
-   *   the name of the volume.
-   * @param bucketName
-   *   the name of the bucket.
-   * @param startKeyName
-   *   the start key name, only the keys whose name is
-   *   after this value will be included in the result.
-   * @param keyPrefix
-   *   key name prefix, only the keys whose name has
-   *   this prefix will be included in the result.
-   * @param maxKeys
-   *   the maximum number of keys to return. It ensures
-   *   the size of the result will not exceed this limit.
-   * @return a list of keys.
-   * @throws IOException
-   */
-  List<KsmKeyInfo> listKeys(String volumeName,
-      String bucketName, String startKeyName, String keyPrefix, int maxKeys)
-      throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43d38114/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocol/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocol/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocol/package-info.java
deleted file mode 100644
index d1d12dc..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ksm/protocol/package-info.java
+++ /dev/null
@@ -1,19 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ksm.protocol;
\ No newline at end of file


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