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/09/18 22:25:15 UTC

[3/3] hadoop git commit: HDFS-12385. Ozone: OzoneClient: Refactoring OzoneClient API. Contributed by Nadakumar.

HDFS-12385. Ozone: OzoneClient: Refactoring OzoneClient API. Contributed by Nadakumar.


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

Branch: refs/heads/HDFS-7240
Commit: e5e7c454d5471f7e4b753b3e3b6714278ca6a595
Parents: 19f7f87
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Mon Sep 18 15:16:03 2017 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Mon Sep 18 15:24:10 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ozone/OzoneConfigKeys.java    |  10 +
 .../apache/hadoop/ozone/client/BucketArgs.java  | 123 ++++
 .../apache/hadoop/ozone/client/ObjectStore.java |  91 +++
 .../apache/hadoop/ozone/client/OzoneBucket.java | 172 +++++-
 .../apache/hadoop/ozone/client/OzoneClient.java | 490 +++-------------
 .../hadoop/ozone/client/OzoneClientFactory.java | 118 +++-
 .../client/OzoneClientInvocationHandler.java    |  62 ++
 .../hadoop/ozone/client/OzoneClientUtils.java   |  27 +-
 .../apache/hadoop/ozone/client/OzoneKey.java    |  52 +-
 .../apache/hadoop/ozone/client/OzoneQuota.java  | 198 +++++++
 .../apache/hadoop/ozone/client/OzoneVolume.java | 169 ++++--
 .../apache/hadoop/ozone/client/VolumeArgs.java  | 128 ++++
 .../ozone/client/protocol/ClientProtocol.java   | 296 ++++++++++
 .../ozone/client/protocol/package-info.java     |  23 +
 .../ozone/client/rest/OzoneRestClient.java      | 510 ----------------
 .../hadoop/ozone/client/rest/RestClient.java    | 209 +++++++
 .../ozone/client/rest/headers/Header.java       |  16 +-
 .../ozone/client/rest/headers/package-info.java |  18 +-
 .../hadoop/ozone/client/rpc/OzoneRpcClient.java | 580 -------------------
 .../hadoop/ozone/client/rpc/RpcClient.java      | 474 +++++++++++++++
 .../org/apache/hadoop/ozone/tools/Corona.java   |  52 +-
 .../src/main/resources/ozone-default.xml        |  11 +
 .../ozone/client/rpc/TestOzoneRpcClient.java    | 259 +++++----
 23 files changed, 2299 insertions(+), 1789 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/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
index 272edaf..efa39a5 100644
--- 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
@@ -20,6 +20,9 @@ package org.apache.hadoop.ozone;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rest.RestClient;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.scm.ScmConfigKeys;
 
 /**
@@ -103,6 +106,13 @@ public final class OzoneConfigKeys {
   public static final String OZONE_ADMINISTRATORS =
       "ozone.administrators";
 
+  public static final String OZONE_CLIENT_PROTOCOL =
+      "ozone.client.protocol";
+  public static final Class<? extends ClientProtocol>
+      OZONE_CLIENT_PROTOCOL_RPC = RpcClient.class;
+  public static final Class<? extends ClientProtocol>
+      OZONE_CLIENT_PROTOCOL_REST = RestClient.class;
+
   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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
new file mode 100644
index 0000000..5d07df2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
@@ -0,0 +1,123 @@
+/**
+ * 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.OzoneAcl;
+
+import java.util.List;
+
+/**
+ * This class encapsulates the arguments that are
+ * required for creating a bucket.
+ */
+public final class BucketArgs {
+
+  /**
+   * 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 isVersionEnabled Bucket version flag.
+   * @param storageType Storage type to be used.
+   * @param acls list of ACLs.
+   */
+  private BucketArgs(Boolean isVersionEnabled, StorageType storageType,
+                     List<OzoneAcl> acls) {
+    this.acls = acls;
+    this.isVersionEnabled = isVersionEnabled;
+    this.storageType = storageType;
+  }
+
+  /**
+   * Returns true if bucket version is enabled, else false.
+   * @return isVersionEnabled
+   */
+  public Boolean isVersionEnabled() {
+    return isVersionEnabled;
+  }
+
+  /**
+   * Returns the type of storage to be used.
+   * @return StorageType
+   */
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Returns the ACL's associated with this bucket.
+   * @return List<OzoneAcl>
+   */
+  public List<OzoneAcl> getAcls() {
+    return acls;
+  }
+
+  /**
+   * Returns new builder class that builds a KsmBucketInfo.
+   *
+   * @return Builder
+   */
+  public static BucketArgs.Builder newBuilder() {
+    return new BucketArgs.Builder();
+  }
+
+  /**
+   * Builder for KsmBucketInfo.
+   */
+  public static class Builder {
+    private Boolean isVersionEnabled;
+    private StorageType storageType;
+    private List<OzoneAcl> acls;
+
+    public BucketArgs.Builder setIsVersionEnabled(Boolean versionFlag) {
+      this.isVersionEnabled = versionFlag;
+      return this;
+    }
+
+    public BucketArgs.Builder setStorageType(StorageType storage) {
+      this.storageType = storage;
+      return this;
+    }
+
+    public BucketArgs.Builder setAcls(List<OzoneAcl> listOfAcls) {
+      this.acls = listOfAcls;
+      return this;
+    }
+
+    /**
+     * Constructs the BucketArgs.
+     * @return instance of BucketArgs.
+     */
+    public BucketArgs build() {
+      return new BucketArgs(isVersionEnabled, storageType, acls);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
new file mode 100644
index 0000000..f8bb21a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
@@ -0,0 +1,91 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+
+import java.io.IOException;
+
+/**
+ * ObjectStore class is responsible for the client operations that can be
+ * performed on Ozone Object Store.
+ */
+public class ObjectStore {
+
+  /**
+   * The proxy used for connecting to the cluster and perform
+   * client operations.
+   */
+  private final ClientProtocol proxy;
+
+  /**
+   * Creates an instance of ObjectStore with the proxy.
+   * @param proxy ClientProtocol proxy
+   */
+  public ObjectStore(ClientProtocol proxy) {
+    this.proxy = proxy;
+  }
+
+  /**
+   * Creates the volume with default values.
+   * @param volumeName Name of the volume to be created.
+   * @throws IOException
+   */
+  public void createVolume(String volumeName) throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    proxy.createVolume(volumeName);
+  }
+
+  /**
+   * Creates the volume.
+   * @param volumeName Name of the volume to be created.
+   * @param volumeArgs Volume properties.
+   * @throws IOException
+   */
+  public void createVolume(String volumeName, VolumeArgs volumeArgs)
+      throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(volumeArgs);
+    proxy.createVolume(volumeName, volumeArgs);
+  }
+
+  /**
+   * Returns the volume information.
+   * @param volumeName Name of the volume.
+   * @return OzoneVolume
+   * @throws IOException
+   */
+  public OzoneVolume getVolume(String volumeName) throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    OzoneVolume volume = proxy.getVolumeDetails(volumeName);
+    volume.setClientProxy(proxy);
+    return volume;
+  }
+
+  /**
+   * Deletes the volume.
+   * @param volumeName Name of the volume.
+   * @throws IOException
+   */
+  public void deleteVolume(String volumeName) throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    proxy.deleteVolume(volumeName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/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
index bfd5714..6d53c5b 100644
--- 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
@@ -1,29 +1,32 @@
-/*
+/**
  * 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
+ * with the License.  You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ *     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.
+ * 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.Preconditions;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.OzoneConsts.Versioning;
 
+import java.io.IOException;
 import java.util.List;
 
 /**
@@ -38,36 +41,53 @@ public class OzoneBucket {
   /**
    * Name of the bucket.
    */
-  private final String bucketName;
+  private final String name;
   /**
    * Bucket ACLs.
    */
-  private final List<OzoneAcl> acls;
+  private List<OzoneAcl> acls;
 
   /**
    * Type of storage to be used for this bucket.
    * [RAM_DISK, SSD, DISK, ARCHIVE]
    */
-  private final StorageType storageType;
+  private StorageType storageType;
 
   /**
    * Bucket Version flag.
    */
-  private final Versioning versioning;
+  private Boolean versioning;
 
+  /**
+   * The proxy used for connecting to the cluster and perform
+   * client operations.
+   */
+  private ClientProtocol proxy;
 
   /**
-   * 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;
+   * Constructs OzoneBucket instance.
+   * @param volumeName Name of the volume the bucket belongs to.
+   * @param bucketName Name of the bucket.
+   * @param acls ACLs associated with the bucket.
+   * @param storageType StorageType of the bucket.
+   * @param versioning versioning status of the bucket.
+   */
+  public OzoneBucket(String volumeName, String bucketName,
+                     List<OzoneAcl> acls, StorageType storageType,
+                     Boolean versioning) {
+    this.volumeName = volumeName;
+    this.name = bucketName;
+    this.acls = acls;
+    this.storageType = storageType;
+    this.versioning = versioning;
+  }
+
+  /**
+   * Sets the proxy using which client operations are performed.
+   * @param clientProxy
+   */
+  public void setClientProxy(ClientProtocol clientProxy) {
+    this.proxy = clientProxy;
   }
 
   /**
@@ -84,8 +104,8 @@ public class OzoneBucket {
    *
    * @return bucketName
    */
-  public String getBucketName() {
-    return bucketName;
+  public String getName() {
+    return name;
   }
 
   /**
@@ -111,8 +131,104 @@ public class OzoneBucket {
    *
    * @return versioning
    */
-  public Versioning getVersioning() {
+  public Boolean getVersioning() {
     return versioning;
   }
 
+  /**
+   * Adds ACLs to the Bucket.
+   * @param addAcls ACLs to be added
+   * @throws IOException
+   */
+  public void addAcls(List<OzoneAcl> addAcls) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(addAcls);
+    proxy.addBucketAcls(volumeName, name, addAcls);
+    addAcls.stream().filter(acl -> !acls.contains(acl)).forEach(
+        acls::add);
+  }
+
+  /**
+   * Removes ACLs from the bucket.
+   * @param removeAcls ACLs to be removed
+   * @throws IOException
+   */
+  public void removeAcls(List<OzoneAcl> removeAcls) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(removeAcls);
+    proxy.removeBucketAcls(volumeName, name, removeAcls);
+    acls.removeAll(removeAcls);
+  }
+
+  /**
+   * Sets/Changes the storage type of the bucket.
+   * @param newStorageType Storage type to be set
+   * @throws IOException
+   */
+  public void setStorageType(StorageType newStorageType) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(newStorageType);
+    proxy.setBucketStorageType(volumeName, name, newStorageType);
+    storageType = newStorageType;
+  }
+
+  /**
+   * Enable/Disable versioning of the bucket.
+   * @param newVersioning
+   * @throws IOException
+   */
+  public void setVersioning(Boolean newVersioning) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(newVersioning);
+    proxy.setBucketVersioning(volumeName, name, newVersioning);
+    versioning = newVersioning;
+  }
+
+  /**
+   * Creates a new key in the bucket.
+   * @param key Name of the key to be created.
+   * @param size Size of the data the key will point to.
+   * @return OzoneOutputStream to which the data has to be written.
+   * @throws IOException
+   */
+  public OzoneOutputStream createKey(String key, long size)throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(key);
+    return proxy.createKey(volumeName, name, key, size);
+  }
+
+  /**
+   * Reads an existing key from the bucket.
+   * @param key Name of the key to be read.
+   * @return OzoneInputStream the stream using which the data can be read.
+   * @throws IOException
+   */
+  public OzoneInputStream readKey(String key) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(key);
+    return proxy.getKey(volumeName, name, key);
+  }
+
+  /**
+   * Returns information about the key.
+   * @param key Name of the key.
+   * @return OzoneKey Information about the key.
+   * @throws IOException
+   */
+  public OzoneKey getKey(String key) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(key);
+    return proxy.getKeyDetails(volumeName, name, key);
+  }
+
+  /**
+   * Deletes key from the bucket.
+   * @param key Name of the key to be deleted.
+   * @throws IOException
+   */
+  public void deleteKey(String key) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(key);
+    proxy.deleteKey(volumeName, name, key);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/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
index a7808d8..b7c3a11 100644
--- 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
@@ -1,415 +1,101 @@
 /**
- * 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>
+ * 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.
+ * 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 org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 
+import java.io.Closeable;
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
 
 /**
- * OzoneClient can connect to a Ozone Cluster and
+ * OzoneClient connects to 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;
+public class OzoneClient implements Closeable {
+
+  /*
+   * OzoneClient connects to Ozone Cluster and
+   * perform basic operations.
+   *
+   * +-------------+     +---+   +-------------------------------------+
+   * | OzoneClient | --> | C |   | Object Store                        |
+   * |_____________|     | l |   |  +-------------------------------+  |
+   *                     | i |   |  | Volume(s)                     |  |
+   *                     | e |   |  |   +------------------------+  |  |
+   *                     | n |   |  |   | Bucket(s)              |  |  |
+   *                     | t |   |  |   |   +------------------+ |  |  |
+   *                     |   |   |  |   |   | Key -> Value (s) | |  |  |
+   *                     | P |-->|  |   |   |                  | |  |  |
+   *                     | r |   |  |   |   |__________________| |  |  |
+   *                     | o |   |  |   |                        |  |  |
+   *                     | t |   |  |   |________________________|  |  |
+   *                     | o |   |  |                               |  |
+   *                     | c |   |  |_______________________________|  |
+   *                     | o |   |                                     |
+   *                     | l |   |_____________________________________|
+   *                     |___|
+   * Example:
+   * ObjectStore store = client.getObjectStore();
+   * store.createVolume(“volume one”, VolumeArgs);
+   * volume.setQuota(“10 GB”);
+   * OzoneVolume volume = store.getVolume(“volume one”);
+   * volume.createBucket(“bucket one”, BucketArgs);
+   * bucket.setVersioning(true);
+   * OzoneOutputStream os = bucket.createKey(“key one”, 1024);
+   * os.write(byte[]);
+   * os.close();
+   * OzoneInputStream is = bucket.readKey(“key one”);
+   * is.read();
+   * is.close();
+   * bucket.deleteKey(“key one”);
+   * volume.deleteBucket(“bucket one”);
+   * store.deleteVolume(“volume one”);
+   * client.close();
+   */
+
+  private final ClientProtocol proxy;
+  private final ObjectStore objectStore;
+
+  /**
+   * Creates a new OzoneClient object, generally constructed
+   * using {@link OzoneClientFactory}.
+   * @param proxy
+   */
+  public OzoneClient(ClientProtocol proxy) {
+    this.proxy = proxy;
+    this.objectStore = new ObjectStore(this.proxy);
+  }
+
+  /**
+   * Returns the object store associated with the Ozone Cluster.
+   * @return ObjectStore
+   */
+  public ObjectStore getObjectStore() {
+    return objectStore;
+  }
+
+  /**
+   * Closes the client and all the underlying resources.
+   * @throws IOException
+   */
+  @Override
+  public void close() throws IOException {
+    proxy.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/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
index b098be9..580cd11 100644
--- 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
@@ -20,64 +20,94 @@ 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 org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 
 import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Proxy;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .OZONE_CLIENT_PROTOCOL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .OZONE_CLIENT_PROTOCOL_REST;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .OZONE_CLIENT_PROTOCOL_RPC;
 
 /**
  * Factory class to create different types of OzoneClients.
+ * Based on <code>ozone.client.protocol</code>, it decides which
+ * protocol to use for the communication.
+ * Default value is
+ * <code>org.apache.hadoop.ozone.client.rpc.RpcClient</code>.<br>
+ * OzoneClientFactory constructs a proxy using
+ * {@link OzoneClientInvocationHandler}
+ * and creates OzoneClient instance with it.
+ * {@link OzoneClientInvocationHandler} dispatches the call to
+ * underlying {@link ClientProtocol} implementation.
  */
 public final class OzoneClientFactory {
 
+  private enum ClientType {
+    RPC, REST
+  }
+
   /**
    * Private constructor, class is not meant to be initialized.
    */
   private OzoneClientFactory(){}
 
+  private static final Logger LOG = LoggerFactory.getLogger(
+      OzoneClientFactory.class);
+
   private static Configuration configuration;
 
   /**
-   * Returns an OzoneClient which will use RPC protocol to perform
-   * client operations.
-   *
+   * Returns an OzoneClient which will use protocol defined through
+   * <code>ozone.client.protocol</code> 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());
+    return getClient(null);
   }
 
   /**
    * 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());
+    return getClient(ClientType.RPC);
   }
 
   /**
-   * Returns an OzoneClient which will use RPC protocol to perform
+   * Returns an OzoneClient which will use REST protocol to perform
    * client operations.
-   *
    * @return OzoneClient
    * @throws IOException
    */
   public static OzoneClient getRestClient() throws IOException {
-    return new OzoneRestClient(getConfiguration());
+    return getClient(ClientType.REST);
   }
 
   /**
-   * Sets the configuration, which will be used while creating OzoneClient.
-   *
-   * @param conf
+   * Returns OzoneClient with protocol type set base on ClientType.
+   * @param clientType
+   * @return OzoneClient
+   * @throws IOException
    */
-  public static void setConfiguration(Configuration conf) {
-    configuration = conf;
+  private static OzoneClient getClient(ClientType clientType)
+      throws IOException {
+    OzoneClientInvocationHandler clientHandler =
+        new OzoneClientInvocationHandler(getProtocolClass(clientType));
+    ClientProtocol proxy = (ClientProtocol) Proxy.newProxyInstance(
+        OzoneClientInvocationHandler.class.getClassLoader(),
+        new Class<?>[]{ClientProtocol.class}, clientHandler);
+    return new OzoneClient(proxy);
   }
 
   /**
@@ -92,4 +122,58 @@ public final class OzoneClientFactory {
     }
     return configuration;
   }
+
+  /**
+   * Based on the clientType, client protocol instance is created.
+   * If clientType is null, <code>ozone.client.protocol</code> property
+   * will be used to decide the protocol to be used.
+   * @param clientType type of client protocol to be created
+   * @return ClientProtocol implementation
+   * @throws IOException
+   */
+  private static ClientProtocol getProtocolClass(ClientType clientType)
+      throws IOException {
+    Class<? extends ClientProtocol> protocolClass = null;
+    if(clientType != null) {
+      switch (clientType) {
+      case RPC:
+        protocolClass = OZONE_CLIENT_PROTOCOL_RPC;
+        break;
+      case REST:
+        protocolClass = OZONE_CLIENT_PROTOCOL_REST;
+        break;
+      default:
+        LOG.warn("Invalid ClientProtocol type, falling back to RPC.");
+        protocolClass = OZONE_CLIENT_PROTOCOL_RPC;
+        break;
+      }
+    } else {
+      protocolClass = (Class<ClientProtocol>)
+          getConfiguration().getClass(
+              OZONE_CLIENT_PROTOCOL, OZONE_CLIENT_PROTOCOL_RPC);
+    }
+    try {
+      Constructor<? extends ClientProtocol> ctor =
+          protocolClass.getConstructor(Configuration.class);
+      return ctor.newInstance(getConfiguration());
+    } catch (Exception e) {
+      final String message = "Couldn't create protocol " + protocolClass;
+      LOG.warn(message, e);
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      } else {
+        throw new IOException(message, e);
+      }
+    }
+  }
+
+  /**
+   * Sets the configuration, which will be used while creating OzoneClient.
+   *
+   * @param conf
+   */
+  public static void setConfiguration(Configuration conf) {
+    configuration = conf;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientInvocationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientInvocationHandler.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientInvocationHandler.java
new file mode 100644
index 0000000..3051e2d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientInvocationHandler.java
@@ -0,0 +1,62 @@
+/**
+ * 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.client.protocol.ClientProtocol;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+/**
+ * Invocation Handler for ozone client which dispatches the call to underlying
+ * ClientProtocol implementation.
+ */
+public class OzoneClientInvocationHandler implements InvocationHandler {
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(OzoneClient.class);
+  private final ClientProtocol target;
+
+  /**
+   * Constructs OzoneClientInvocationHandler with the proxy.
+   * @param target proxy to be used for method invocation.
+   */
+  public OzoneClientInvocationHandler(ClientProtocol target) {
+    this.target = target;
+  }
+
+  @Override
+  public Object invoke(Object proxy, Method method, Object[] args)
+      throws Throwable {
+    LOG.trace("Invoking method {} on proxy {}", method, proxy);
+    try {
+      long startTime = Time.monotonicNow();
+      Object result = method.invoke(target, args);
+      LOG.debug("Call: {} took {} ms", method,
+          Time.monotonicNow() - startTime);
+      return result;
+    } catch(InvocationTargetException iEx) {
+      throw iEx.getCause();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/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
index 9390a85..cc3632d 100644
--- 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
@@ -1,18 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
+ * 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.
+ * 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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/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
index 368736a..d6a591f 100644
--- 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
@@ -1,28 +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
+ * with the License.  You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ *     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.
+ * 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.
  */
@@ -39,27 +34,22 @@ public class OzoneKey {
   /**
    * Name of the Key.
    */
-  private final String keyName;
+  private final String name;
   /**
    * 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();
+  public OzoneKey(String volumeName, String bucketName,
+                  String keyName, long size) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.name = keyName;
+    this.dataSize = size;
   }
 
   /**
@@ -85,8 +75,8 @@ public class OzoneKey {
    *
    * @return keyName
    */
-  public String getKeyName() {
-    return keyName;
+  public String getName() {
+    return name;
   }
 
   /**
@@ -98,12 +88,4 @@ public class OzoneKey {
     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/e5e7c454/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneQuota.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneQuota.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneQuota.java
new file mode 100644
index 0000000..bff529b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneQuota.java
@@ -0,0 +1,198 @@
+/**
+ * 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.OzoneConsts;
+
+
+/**
+ * represents an OzoneQuota Object that can be applied to
+ * a storage volume.
+ */
+public class OzoneQuota {
+
+  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";
+
+  private Units unit;
+  private int size;
+
+  /** Quota Units.*/
+  public enum Units {UNDEFINED, BYTES, MB, GB, TB}
+
+  /**
+   * Returns size.
+   *
+   * @return int
+   */
+  public int getSize() {
+    return size;
+  }
+
+  /**
+   * Returns Units.
+   *
+   * @return Unit in MB, GB or TB
+   */
+  public Units getUnit() {
+    return unit;
+  }
+
+  /**
+   * Constructs a default Quota object.
+   */
+  public OzoneQuota() {
+    this.size = 0;
+    this.unit = Units.UNDEFINED;
+  }
+
+  /**
+   * Constructor for Ozone Quota.
+   *
+   * @param size - Integer Size
+   * @param unit MB, GB  or TB
+   */
+  public OzoneQuota(int size, Units unit) {
+    this.size = size;
+    this.unit = unit;
+  }
+
+  /**
+   * Formats a quota as a string.
+   *
+   * @param quota the quota to format
+   * @return string representation of quota
+   */
+  public static String formatQuota(OzoneQuota quota) {
+    return String.valueOf(quota.size) + quota.unit;
+  }
+
+  /**
+   * Parses a user provided string and returns the
+   * Quota Object.
+   *
+   * @param quotaString Quota String
+   *
+   * @return OzoneQuota object
+   *
+   * @throws IllegalArgumentException
+   */
+  public static OzoneQuota parseQuota(String quotaString)
+      throws IllegalArgumentException {
+
+    if ((quotaString == null) || (quotaString.isEmpty())) {
+      throw new IllegalArgumentException(
+          "Quota string cannot be null or empty.");
+    }
+
+    String uppercase = quotaString.toUpperCase().replaceAll("\\s+", "");
+    String size = "";
+    int nSize;
+    Units currUnit = Units.MB;
+    Boolean found = false;
+    if (uppercase.endsWith(OZONE_QUOTA_MB)) {
+      size = uppercase
+          .substring(0, uppercase.length() - OZONE_QUOTA_MB.length());
+      currUnit = Units.MB;
+      found = true;
+    }
+
+    if (uppercase.endsWith(OZONE_QUOTA_GB)) {
+      size = uppercase
+          .substring(0, uppercase.length() - OZONE_QUOTA_GB.length());
+      currUnit = Units.GB;
+      found = true;
+    }
+
+    if (uppercase.endsWith(OZONE_QUOTA_TB)) {
+      size = uppercase
+          .substring(0, uppercase.length() - OZONE_QUOTA_TB.length());
+      currUnit = Units.TB;
+      found = true;
+    }
+
+    if (uppercase.endsWith(OZONE_QUOTA_BYTES)) {
+      size = uppercase
+          .substring(0, uppercase.length() - OZONE_QUOTA_BYTES.length());
+      currUnit = Units.BYTES;
+      found = true;
+    }
+
+    if (!found) {
+      throw new IllegalArgumentException(
+          "Quota unit not recognized. Supported values are BYTES, MB, GB and " +
+              "TB.");
+    }
+
+    nSize = Integer.parseInt(size);
+    if (nSize < 0) {
+      throw new IllegalArgumentException("Quota cannot be negative.");
+    }
+
+    return new OzoneQuota(nSize, currUnit);
+  }
+
+
+  /**
+   * Returns size in Bytes or -1 if there is no Quota.
+   */
+  public long sizeInBytes() {
+    switch (this.unit) {
+    case BYTES:
+      return this.getSize();
+    case MB:
+      return this.getSize() * OzoneConsts.MB;
+    case GB:
+      return this.getSize() * OzoneConsts.GB;
+    case TB:
+      return this.getSize() * OzoneConsts.TB;
+    case UNDEFINED:
+    default:
+      return -1;
+    }
+  }
+
+  /**
+   * Returns OzoneQuota corresponding to size in bytes.
+   *
+   * @param sizeInBytes size in bytes to be converted
+   *
+   * @return OzoneQuota object
+   */
+  public static OzoneQuota getOzoneQuota(long sizeInBytes) {
+    long size;
+    Units unit;
+    if (sizeInBytes % OzoneConsts.TB == 0) {
+      size = sizeInBytes / OzoneConsts.TB;
+      unit = Units.TB;
+    } else if (sizeInBytes % OzoneConsts.GB == 0) {
+      size = sizeInBytes / OzoneConsts.GB;
+      unit = Units.GB;
+    } else if (sizeInBytes % OzoneConsts.MB == 0) {
+      size = sizeInBytes / OzoneConsts.MB;
+      unit = Units.MB;
+    } else {
+      size = sizeInBytes;
+      unit = Units.BYTES;
+    }
+    return new OzoneQuota((int)size, unit);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/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
index 9c2ec3d..575fb25 100644
--- 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
@@ -1,25 +1,29 @@
-/*
+/**
  * 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
+ * with the License.  You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ *     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.
+ * 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;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+
+import java.io.IOException;
+import java.util.List;
 
 /**
  * A class that encapsulates OzoneVolume.
@@ -27,64 +31,70 @@ import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
 public class OzoneVolume {
 
   /**
-   * Admin Name of the Volume.
+   * Name of the Volume.
    */
-  private final String adminName;
+  private final String name;
+
   /**
-   * Owner of the Volume.
+   * Admin Name of the Volume.
    */
-  private final String ownerName;
+  private String admin;
   /**
-   * Name of the Volume.
+   * Owner of the Volume.
    */
-  private final String volumeName;
+  private String owner;
   /**
    * Quota allocated for the Volume.
    */
-  private final long quotaInBytes;
+  private long quotaInBytes;
   /**
    * Volume ACLs.
    */
-  private final KsmOzoneAclMap aclMap;
+  private List<OzoneAcl> acls;
+
+  private ClientProtocol proxy;
 
   /**
-   * 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();
+   * Constructs OzoneVolume.
+   */
+  public OzoneVolume(String name, String admin, String owner,
+                     long quotaInBytes, List<OzoneAcl> acls) {
+    this.name = name;
+    this.admin = admin;
+    this.owner = owner;
+    this.quotaInBytes = quotaInBytes;
+    this.acls = acls;
+  }
+
+  public void setClientProxy(ClientProtocol clientProxy) {
+    this.proxy = clientProxy;
   }
 
   /**
-   * Returns Volume's admin name.
+   * Returns Volume name.
    *
-   * @return adminName
+   * @return volumeName
    */
-  public String getAdminName() {
-    return adminName;
+  public String getName() {
+    return name;
   }
 
   /**
-   * Returns Volume's owner name.
+   * Returns Volume's admin name.
    *
-   * @return ownerName
+   * @return adminName
    */
-  public String getOwnerName() {
-    return ownerName;
+  public String getAdmin() {
+    return admin;
   }
 
   /**
-   * Returns Volume name.
+   * Returns Volume's owner name.
    *
-   * @return volumeName
+   * @return ownerName
    */
-  public String getVolumeName() {
-    return volumeName;
+  public String getOwner() {
+    return owner;
   }
 
   /**
@@ -101,7 +111,82 @@ public class OzoneVolume {
    *
    * @return aclMap
    */
-  public KsmOzoneAclMap getAclMap() {
-    return aclMap;
+  public List<OzoneAcl> getAcls() {
+    return acls;
+  }
+
+  /**
+   * Sets/Changes the owner of this Volume.
+   * @param owner new owner
+   * @throws IOException
+   */
+  public void setOwner(String owner) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(owner);
+    proxy.setVolumeOwner(name, owner);
+    this.owner = owner;
+  }
+
+  /**
+   * Sets/Changes the quota of this Volume.
+   * @param quota new quota
+   * @throws IOException
+   */
+  public void setQuota(OzoneQuota quota) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(quota);
+    proxy.setVolumeQuota(name, quota);
+    this.quotaInBytes = quota.sizeInBytes();
+  }
+
+  /**
+   * Creates a new Bucket in this Volume, with default values.
+   * @param bucketName Name of the Bucket
+   * @throws IOException
+   */
+  public void createBucket(String bucketName)
+      throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(bucketName);
+    proxy.createBucket(name, bucketName);
+  }
+
+  /**
+   * Creates a new Bucket in this Volume, with properties set in bucketArgs.
+   * @param bucketName Name of the Bucket
+   * @param bucketArgs Properties to be set
+   * @throws IOException
+   */
+  public void createBucket(String bucketName, BucketArgs bucketArgs)
+      throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(bucketArgs);
+    proxy.createBucket(name, bucketName, bucketArgs);
+  }
+
+  /**
+   * Get the Bucket from this Volume.
+   * @param bucketName Name of the Bucket
+   * @return OzoneBucket
+   * @throws IOException
+   */
+  public OzoneBucket getBucket(String bucketName) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(bucketName);
+    OzoneBucket bucket = proxy.getBucketDetails(name, bucketName);
+    bucket.setClientProxy(proxy);
+    return bucket;
+  }
+
+  /**
+   * Deletes the Bucket from this Volume.
+   * @param bucketName Name of the Bucket
+   * @throws IOException
+   */
+  public void deleteBucket(String bucketName) throws IOException {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
+    Preconditions.checkNotNull(bucketName);
+    proxy.deleteBucket(name, bucketName);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java
new file mode 100644
index 0000000..f1aa031
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java
@@ -0,0 +1,128 @@
+/**
+ * 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.OzoneAcl;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This class encapsulates the arguments that are
+ * required for creating a volume.
+ */
+public final class VolumeArgs {
+
+  private final String admin;
+  private final String owner;
+  private final String quota;
+  private final List<OzoneAcl> acls;
+
+  /**
+   * Private constructor, constructed via builder.
+   * @param admin Administrator's name.
+   * @param owner Volume owner's name
+   * @param quota Volume Quota.
+   * @param acls User to access rights map.
+   */
+  private VolumeArgs(String admin, String owner,
+                        String quota, List<OzoneAcl> acls) {
+    this.admin = admin;
+    this.owner = owner;
+    this.quota = quota;
+    this.acls = acls;
+  }
+
+  /**
+   * Returns the Admin Name.
+   * @return String.
+   */
+  public String getAdmin() {
+    return admin;
+  }
+
+  /**
+   * Returns the owner Name.
+   * @return String
+   */
+  public String getOwner() {
+    return owner;
+  }
+
+  /**
+   * Returns Volume Quota.
+   * @return Quota.
+   */
+  public String getQuota() {
+    return quota;
+  }
+
+  public List<OzoneAcl> getAcls() {
+    return acls;
+  }
+  /**
+   * Returns new builder class that builds a KsmVolumeArgs.
+   *
+   * @return Builder
+   */
+  public static VolumeArgs.Builder newBuilder() {
+    return new VolumeArgs.Builder();
+  }
+
+  /**
+   * Builder for KsmVolumeArgs.
+   */
+  public static class Builder {
+    private String adminName;
+    private String ownerName;
+    private String volumeQuota;
+    private List<OzoneAcl> listOfAcls;
+
+
+    public VolumeArgs.Builder setAdmin(String admin) {
+      this.adminName = admin;
+      return this;
+    }
+
+    public VolumeArgs.Builder setOwner(String owner) {
+      this.ownerName = owner;
+      return this;
+    }
+
+    public VolumeArgs.Builder setQuota(String quota) {
+      this.volumeQuota = quota;
+      return this;
+    }
+
+    public VolumeArgs.Builder setAcls(List<OzoneAcl> acls)
+        throws IOException {
+      this.listOfAcls = acls;
+      return this;
+    }
+
+    /**
+     * Constructs a CreateVolumeArgument.
+     * @return CreateVolumeArgs.
+     */
+    public VolumeArgs build() {
+      return new VolumeArgs(adminName, ownerName, volumeQuota, listOfAcls);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5e7c454/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
new file mode 100644
index 0000000..54aa3f9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -0,0 +1,296 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneQuota;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+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;
+
+/**
+ * An implementer of this interface is capable of connecting to Ozone Cluster
+ * and perform client operations. The protocol used for communication is
+ * determined by the implementation class specified by
+ * property <code>ozone.client.protocol</code>. The build-in implementation
+ * includes: {@link org.apache.hadoop.ozone.client.rpc.RpcClient} for RPC and
+ * {@link  org.apache.hadoop.ozone.client.rest.RestClient} for REST.
+ */
+public interface ClientProtocol {
+
+  /**
+   * Creates a new Volume.
+   * @param volumeName Name of the Volume
+   * @throws IOException
+   */
+  void createVolume(String volumeName)
+      throws IOException;
+
+  /**
+   * Creates a new Volume with properties set in VolumeArgs.
+   * @param volumeName Name of the Volume
+   * @param args Properties to be set for the Volume
+   * @throws IOException
+   */
+  void createVolume(String volumeName, VolumeArgs args)
+      throws IOException;
+
+  /**
+   * Sets the owner of 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, OzoneQuota quota)
+      throws IOException;
+
+  /**
+   * Returns {@link OzoneVolume}.
+   * @param volumeName Name of the Volume
+   * @return {@link OzoneVolume}
+   * @throws IOException
+   * */
+  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 {@link OzoneVolume} 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  {@link OzoneVolume} 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 properties set in BucketArgs.
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param bucketArgs Bucket Arguments
+   * @throws IOException
+   */
+  void createBucket(String volumeName, String bucketName,
+                    BucketArgs bucketArgs)
+      throws IOException;
+
+  /**
+   * Adds ACLs to the Bucket.
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param addAcls ACLs to be added
+   * @throws IOException
+   */
+  void addBucketAcls(String volumeName, String bucketName,
+                     List<OzoneAcl> addAcls)
+      throws IOException;
+
+  /**
+   * Removes ACLs from a Bucket.
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param removeAcls ACLs to be removed
+   * @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
+   * @param versioning True to enable Versioning, False to disable.
+   * @throws IOException
+   */
+  void setBucketVersioning(String volumeName, String bucketName,
+                           Boolean versioning)
+      throws IOException;
+
+  /**
+   * Sets the Storage Class of a Bucket.
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param storageType StorageType to be set
+   * @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
+   * @param bucketName Name of the Bucket
+   * @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 {@link 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 {@link OzoneBucket} 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 keyName Name of the Key
+   * @param size Size of the data
+   * @return {@link OzoneOutputStream}
+   *
+   */
+  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
+   * @param keyName Name of the Key
+   * @return {@link OzoneInputStream}
+   * @throws IOException
+   */
+  OzoneInputStream getKey(String volumeName, String bucketName, String keyName)
+      throws IOException;
+
+
+  /**
+   * Deletes an existing key.
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param keyName Name of the Key
+   * @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 {@link OzoneKey} Iterator
+   * @throws IOException
+   */
+  Iterator<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 {@link 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/e5e7c454/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/package-info.java
new file mode 100644
index 0000000..f4890a1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/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.protocol;
+
+/**
+ * This package contains Ozone client protocol library classes.
+ */


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