Bläddra i källkod

HDDS-18. Ozone Shell should use RestClient and RpcClient. Contributed by Lokesh Jain.

Anu Engineer 7 år sedan
förälder
incheckning
46e0f27862
38 ändrade filer med 747 tillägg och 523 borttagningar
  1. 11 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
  2. 1 0
      hadoop-ozone/acceptance-test/src/test/compose/docker-compose.yaml
  3. 7 6
      hadoop-ozone/acceptance-test/src/test/robotframework/acceptance/ozone.robot
  4. 29 20
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
  5. 20 5
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
  6. 5 5
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientException.java
  7. 1 1
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java
  8. 86 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
  9. 21 5
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
  10. 4 4
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
  11. 24 23
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
  12. 22 21
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java
  13. 12 11
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
  14. 207 91
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
  15. 2 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
  16. 2 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
  17. 43 16
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
  18. 11 21
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java
  19. 5 15
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
  20. 8 17
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
  21. 25 26
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
  22. 21 20
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java
  23. 6 20
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/DeleteKeyHandler.java
  24. 29 17
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java
  25. 10 21
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
  26. 24 24
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
  27. 33 16
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java
  28. 18 10
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
  29. 4 14
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
  30. 8 30
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
  31. 28 30
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
  32. 8 16
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
  33. 0 6
      hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/Constants.java
  34. 2 1
      hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java
  35. 2 2
      hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
  36. 2 1
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
  37. 3 2
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java
  38. 3 3
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java

+ 11 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java

@@ -58,6 +58,10 @@ public final class OzoneConsts {
   public static final String OZONE_USER = "user";
   public static final String OZONE_USER = "user";
   public static final String OZONE_REQUEST = "request";
   public static final String OZONE_REQUEST = "request";
 
 
+  public static final String OZONE_URI_SCHEME = "o3";
+  public static final String OZONE_HTTP_SCHEME = "http";
+  public static final String OZONE_URI_DELIMITER = "/";
+
   public static final String CONTAINER_EXTENSION = ".container";
   public static final String CONTAINER_EXTENSION = ".container";
   public static final String CONTAINER_META = ".meta";
   public static final String CONTAINER_META = ".meta";
 
 
@@ -95,7 +99,13 @@ public final class OzoneConsts {
   /**
   /**
    * Supports Bucket Versioning.
    * Supports Bucket Versioning.
    */
    */
-  public enum Versioning {NOT_DEFINED, ENABLED, DISABLED}
+  public enum Versioning {
+    NOT_DEFINED, ENABLED, DISABLED;
+
+    public static Versioning getVersioning(boolean versioning) {
+      return versioning ? ENABLED : DISABLED;
+    }
+  }
 
 
   /**
   /**
    * Ozone handler types.
    * Ozone handler types.

+ 1 - 0
hadoop-ozone/acceptance-test/src/test/compose/docker-compose.yaml

@@ -39,6 +39,7 @@ services:
         - ./docker-config
         - ./docker-config
    ksm:
    ksm:
       image: apache/hadoop-runner
       image: apache/hadoop-runner
+      hostname: ksm
       volumes:
       volumes:
          - ${HADOOPDIR}:/opt/hadoop
          - ${HADOOPDIR}:/opt/hadoop
       ports:
       ports:

+ 7 - 6
hadoop-ozone/acceptance-test/src/test/robotframework/acceptance/ozone.robot

@@ -49,14 +49,14 @@ Test rest interface
                     Should contain      ${result}       200 OK
                     Should contain      ${result}       200 OK
 
 
 Test ozone cli
 Test ozone cli
-                    Execute on          datanode        ozone oz -createVolume http://localhost:9880/hive -user bilbo -quota 100TB -root
-    ${result} =     Execute on          datanode        ozone oz -listVolume http://localhost:9880/ -user bilbo | grep -v Removed | jq '.[] | select(.volumeName=="hive")'
+                    Execute on          datanode        ozone oz -createVolume http://ksm/hive -user bilbo -quota 100TB -root
+    ${result} =     Execute on          datanode        ozone oz -listVolume o3://ksm -user bilbo | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.volumeName=="hive")'
                     Should contain      ${result}       createdOn
                     Should contain      ${result}       createdOn
-                    Execute on          datanode        ozone oz -createBucket http://localhost:9880/hive/bb1
-    ${result}       Execute on          datanode        ozone oz -listBucket http://localhost:9880/hive/ | grep -v Removed | jq -r '.[] | select(.bucketName=="bb1") | .volumeName'
+                    Execute on          datanode        ozone oz -createBucket http://ksm/hive/bb1
+    ${result}       Execute on          datanode        ozone oz -listBucket o3://ksm/hive/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.bucketName=="bb1") | .volumeName'
                     Should Be Equal     ${result}       hive
                     Should Be Equal     ${result}       hive
-                    Execute on          datanode        ozone oz -deleteBucket http://localhost:9880/hive/bb1
-                    Execute on          datanode        ozone oz -deleteVolume http://localhost:9880/hive -user bilbo
+                    Execute on          datanode        ozone oz -deleteBucket http://ksm/hive/bb1
+                    Execute on          datanode        ozone oz -deleteVolume http://ksm/hive -user bilbo
 
 
 
 
 
 
@@ -106,6 +106,7 @@ Scale datanodes up
 Execute on
 Execute on
     [arguments]     ${componentname}    ${command}
     [arguments]     ${componentname}    ${command}
     ${rc}           ${return} =         Run docker compose          exec ${componentname} ${command}
     ${rc}           ${return} =         Run docker compose          exec ${componentname} ${command}
+    Log             ${return}
     [return]        ${return}
     [return]        ${return}
 
 
 Run docker compose
 Run docker compose

+ 29 - 20
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java

@@ -106,25 +106,42 @@ public class ObjectStore {
    */
    */
   public Iterator<OzoneVolume> listVolumes(String volumePrefix)
   public Iterator<OzoneVolume> listVolumes(String volumePrefix)
       throws IOException {
       throws IOException {
-    return new VolumeIterator(volumePrefix);
+    return listVolumes(volumePrefix, null);
   }
   }
 
 
   /**
   /**
-   * Returns Iterator to iterate over the List of volumes owned by a specific
-   * user. The result can be restricted using volume prefix, will return all
-   * volumes if volume prefix is null. If user is null, returns the volume of
-   * current user.
+   * Returns Iterator to iterate over all the volumes after prevVolume in object
+   * store. If prevVolume is null it iterates from the first volume.
+   * The result can be restricted using volume prefix, will return all
+   * volumes if volume prefix is null.
+   *
+   * @param volumePrefix Volume prefix to match
+   * @param prevVolume Volumes will be listed after this volume name
+   * @return {@code Iterator<OzoneVolume>}
+   */
+  public Iterator<OzoneVolume> listVolumes(String volumePrefix,
+      String prevVolume) throws IOException {
+    return new VolumeIterator(null, volumePrefix, prevVolume);
+  }
+
+  /**
+   * Returns Iterator to iterate over the list of volumes after prevVolume owned
+   * by a specific user. The result can be restricted using volume prefix, will
+   * return all volumes if volume prefix is null. If user is not null, returns
+   * the volume of current user.
    *
    *
    * @param user User Name
    * @param user User Name
    * @param volumePrefix Volume prefix to match
    * @param volumePrefix Volume prefix to match
+   * @param prevVolume Volumes will be listed after this volume name
    * @return {@code Iterator<OzoneVolume>}
    * @return {@code Iterator<OzoneVolume>}
    */
    */
-  public Iterator<OzoneVolume> listVolumes(String user, String volumePrefix)
+  public Iterator<OzoneVolume> listVolumesByUser(String user,
+      String volumePrefix, String prevVolume)
       throws IOException {
       throws IOException {
     if(Strings.isNullOrEmpty(user)) {
     if(Strings.isNullOrEmpty(user)) {
       user = UserGroupInformation.getCurrentUser().getShortUserName();
       user = UserGroupInformation.getCurrentUser().getShortUserName();
     }
     }
-    return new VolumeIterator(user, volumePrefix);
+    return new VolumeIterator(user, volumePrefix, prevVolume);
   }
   }
 
 
   /**
   /**
@@ -150,25 +167,17 @@ public class ObjectStore {
     private OzoneVolume currentValue;
     private OzoneVolume currentValue;
 
 
     /**
     /**
-     * Creates an Iterator to iterate over all volumes in the cluster,
-     * which matches the volume prefix.
-     * @param volPrefix prefix to match
-     */
-    VolumeIterator(String volPrefix) {
-      this(null, volPrefix);
-    }
-
-    /**
-     * Creates an Iterator to iterate over all volumes of the user,
-     * which matches volume prefix.
+     * Creates an Iterator to iterate over all volumes after prevVolume of the user.
+     * If prevVolume is null it iterates from the first volume. The returned volumes
+     * match volume prefix.
      * @param user user name
      * @param user user name
      * @param volPrefix volume prefix to match
      * @param volPrefix volume prefix to match
      */
      */
-    VolumeIterator(String user, String volPrefix) {
+    VolumeIterator(String user, String volPrefix, String prevVolume) {
       this.user = user;
       this.user = user;
       this.volPrefix = volPrefix;
       this.volPrefix = volPrefix;
       this.currentValue = null;
       this.currentValue = null;
-      this.currentIterator = getNextListOfVolumes(null).iterator();
+      this.currentIterator = getNextListOfVolumes(prevVolume).iterator();
     }
     }
 
 
     @Override
     @Override

+ 20 - 5
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java

@@ -291,7 +291,21 @@ public class OzoneBucket {
    * @return {@code Iterator<OzoneKey>}
    * @return {@code Iterator<OzoneKey>}
    */
    */
   public Iterator<OzoneKey> listKeys(String keyPrefix) {
   public Iterator<OzoneKey> listKeys(String keyPrefix) {
-    return new KeyIterator(keyPrefix);
+    return listKeys(keyPrefix, null);
+  }
+
+  /**
+   * Returns Iterator to iterate over all keys after prevKey in the bucket.
+   * If prevKey is null it iterates from the first key in the bucket.
+   * The result can be restricted using key prefix, will return all
+   * keys if key prefix is null.
+   *
+   * @param keyPrefix Bucket prefix to match
+   * @param prevKey Keys will be listed after this key name
+   * @return {@code Iterator<OzoneKey>}
+   */
+  public Iterator<OzoneKey> listKeys(String keyPrefix, String prevKey) {
+    return new KeyIterator(keyPrefix, prevKey);
   }
   }
 
 
   /**
   /**
@@ -325,14 +339,15 @@ public class OzoneBucket {
 
 
 
 
     /**
     /**
-     * Creates an Iterator to iterate over all keys in the bucket,
-     * which matches volume prefix.
+     * Creates an Iterator to iterate over all keys after prevKey in the bucket.
+     * If prevKey is null it iterates from the first key in the bucket.
+     * The returned keys match key prefix.
      * @param keyPrefix
      * @param keyPrefix
      */
      */
-    KeyIterator(String keyPrefix) {
+    KeyIterator(String keyPrefix, String prevKey) {
       this.keyPrefix = keyPrefix;
       this.keyPrefix = keyPrefix;
       this.currentValue = null;
       this.currentValue = null;
-      this.currentIterator = getNextListOfKeys(null).iterator();
+      this.currentIterator = getNextListOfKeys(prevKey).iterator();
     }
     }
 
 
     @Override
     @Override

+ 5 - 5
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClientException.java → hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientException.java

@@ -15,20 +15,20 @@
  *  See the License for the specific language governing permissions and
  *  See the License for the specific language governing permissions and
  *  limitations under the License.
  *  limitations under the License.
  */
  */
-package org.apache.hadoop.ozone.web.client;
+package org.apache.hadoop.ozone.client;
 
 
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 
 
 /**
 /**
  * This exception is thrown by the Ozone Clients.
  * This exception is thrown by the Ozone Clients.
  */
  */
-public class OzoneRestClientException extends OzoneException {
+public class OzoneClientException extends OzoneException {
   /**
   /**
    * Constructor that allows the shortMessage.
    * Constructor that allows the shortMessage.
    *
    *
    * @param shortMessage Short Message
    * @param shortMessage Short Message
    */
    */
-  public OzoneRestClientException(String shortMessage) {
+  public OzoneClientException(String shortMessage) {
     super(0, shortMessage, shortMessage);
     super(0, shortMessage, shortMessage);
   }
   }
 
 
@@ -38,7 +38,7 @@ public class OzoneRestClientException extends OzoneException {
    * @param shortMessage short message
    * @param shortMessage short message
    * @param ex exception
    * @param ex exception
    */
    */
-  public OzoneRestClientException(String shortMessage, Exception ex) {
+  public OzoneClientException(String shortMessage, Exception ex) {
     super(0, shortMessage, shortMessage, ex);
     super(0, shortMessage, shortMessage, ex);
   }
   }
 
 
@@ -48,7 +48,7 @@ public class OzoneRestClientException extends OzoneException {
    * @param shortMessage Short Message
    * @param shortMessage Short Message
    * @param message long error message
    * @param message long error message
    */
    */
-  public OzoneRestClientException(String shortMessage, String message) {
+  public OzoneClientException(String shortMessage, String message) {
     super(0, shortMessage, message);
     super(0, shortMessage, message);
   }
   }
 }
 }

+ 1 - 1
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java

@@ -292,7 +292,7 @@ public final class OzoneClientFactory {
       return ctor.newInstance(config);
       return ctor.newInstance(config);
     } catch (Exception e) {
     } catch (Exception e) {
       final String message = "Couldn't create protocol " + protocolClass;
       final String message = "Couldn't create protocol " + protocolClass;
-      LOG.error(message + " exception:" + e);
+      LOG.error(message + " exception: ", e);
       if (e.getCause() instanceof IOException) {
       if (e.getCause() instanceof IOException) {
         throw (IOException) e.getCause();
         throw (IOException) e.getCause();
       } else if (e instanceof InvocationTargetException) {
       } else if (e instanceof InvocationTargetException) {

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

@@ -0,0 +1,86 @@
+/**
+ * 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.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
+import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
+import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
+import org.apache.hadoop.ozone.client.rest.response.VolumeOwner;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+
+import static org.apache.hadoop.ozone.web.utils.OzoneUtils.formatTime;
+
+/** A utility class for OzoneClient. */
+public final class OzoneClientUtils {
+
+  private OzoneClientUtils() {}
+
+  /**
+   * Returns a BucketInfo object constructed using fields of the input
+   * OzoneBucket object.
+   *
+   * @param bucket OzoneBucket instance from which BucketInfo object needs to
+   *               be created.
+   * @return BucketInfo instance
+   */
+  public static BucketInfo asBucketInfo(OzoneBucket bucket) {
+    BucketInfo bucketInfo =
+        new BucketInfo(bucket.getVolumeName(), bucket.getName());
+    bucketInfo.setCreatedOn(OzoneUtils.formatTime(bucket.getCreationTime()));
+    bucketInfo.setStorageType(bucket.getStorageType());
+    bucketInfo.setVersioning(
+        OzoneConsts.Versioning.getVersioning(bucket.getVersioning()));
+    return bucketInfo;
+  }
+
+  /**
+   * Returns a VolumeInfo object constructed using fields of the input
+   * OzoneVolume object.
+   *
+   * @param volume OzoneVolume instance from which VolumeInfo object needs to
+   *               be created.
+   * @return VolumeInfo instance
+   */
+  public static VolumeInfo asVolumeInfo(OzoneVolume volume) {
+    VolumeInfo volumeInfo =
+        new VolumeInfo(volume.getName(), formatTime(volume.getCreationTime()),
+            volume.getOwner());
+    volumeInfo.setQuota(OzoneQuota.getOzoneQuota(volume.getQuota()));
+    volumeInfo.setOwner(new VolumeOwner(volume.getOwner()));
+    return volumeInfo;
+  }
+
+  /**
+   * Returns a KeyInfo object constructed using fields of the input
+   * OzoneKey object.
+   *
+   * @param key OzoneKey instance from which KeyInfo object needs to
+   *            be created.
+   * @return KeyInfo instance
+   */
+  public static KeyInfo asKeyInfo(OzoneKey key) {
+    KeyInfo keyInfo = new KeyInfo();
+    keyInfo.setKeyName(key.getName());
+    keyInfo.setCreatedOn(formatTime(key.getCreationTime()));
+    keyInfo.setModifiedOn(formatTime(key.getModificationTime()));
+    keyInfo.setSize(key.getDataSize());
+    return keyInfo;
+  }
+}

+ 21 - 5
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java

@@ -222,7 +222,22 @@ public class OzoneVolume {
    * @return {@code Iterator<OzoneBucket>}
    * @return {@code Iterator<OzoneBucket>}
    */
    */
   public Iterator<OzoneBucket> listBuckets(String bucketPrefix) {
   public Iterator<OzoneBucket> listBuckets(String bucketPrefix) {
-    return new BucketIterator(bucketPrefix);
+    return listBuckets(bucketPrefix, null);
+  }
+
+  /**
+   * Returns Iterator to iterate over all buckets after prevBucket in the volume.
+   * If prevBucket is null it iterates from the first bucket in the volume.
+   * The result can be restricted using bucket prefix, will return all
+   * buckets if bucket prefix is null.
+   *
+   * @param bucketPrefix Bucket prefix to match
+   * @param prevBucket Buckets are listed after this bucket
+   * @return {@code Iterator<OzoneBucket>}
+   */
+  public Iterator<OzoneBucket> listBuckets(String bucketPrefix,
+      String prevBucket) {
+    return new BucketIterator(bucketPrefix, prevBucket);
   }
   }
 
 
   /**
   /**
@@ -250,14 +265,15 @@ public class OzoneVolume {
 
 
 
 
     /**
     /**
-     * Creates an Iterator to iterate over all buckets in the volume,
-     * which matches volume prefix.
+     * Creates an Iterator to iterate over all buckets after prevBucket in the volume.
+     * If prevBucket is null it iterates from the first bucket in the volume.
+     * The returned buckets match bucket prefix.
      * @param bucketPrefix
      * @param bucketPrefix
      */
      */
-    BucketIterator(String bucketPrefix) {
+    public BucketIterator(String bucketPrefix, String prevBucket) {
       this.bucketPrefix = bucketPrefix;
       this.bucketPrefix = bucketPrefix;
       this.currentValue = null;
       this.currentValue = null;
-      this.currentIterator = getNextListOfBuckets(null).iterator();
+      this.currentIterator = getNextListOfBuckets(prevBucket).iterator();
     }
     }
 
 
     @Override
     @Override

+ 4 - 4
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java

@@ -44,7 +44,6 @@ import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
 import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
 import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
 import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
 import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
 import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
-import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
 import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
 import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.protocol.proto
 import org.apache.hadoop.ozone.protocol.proto
@@ -94,7 +93,7 @@ import static java.net.HttpURLConnection.HTTP_OK;
 public class RestClient implements ClientProtocol {
 public class RestClient implements ClientProtocol {
 
 
   private static final String PATH_SEPARATOR = "/";
   private static final String PATH_SEPARATOR = "/";
-  private static final Logger LOG = LoggerFactory.getLogger(RpcClient.class);
+  private static final Logger LOG = LoggerFactory.getLogger(RestClient.class);
 
 
   private final Configuration conf;
   private final Configuration conf;
   private final URI ozoneRestUri;
   private final URI ozoneRestUri;
@@ -195,8 +194,9 @@ public class RestClient implements ClientProtocol {
 
 
       ServiceInfo restServer = selector.getRestServer(dataNodeInfos);
       ServiceInfo restServer = selector.getRestServer(dataNodeInfos);
 
 
-      return NetUtils.createSocketAddr(restServer.getHostname() + ":" +
-          restServer.getPort(ServicePort.Type.HTTP));
+      return NetUtils.createSocketAddr(
+          NetUtils.normalizeHostName(restServer.getHostname()) + ":"
+              + restServer.getPort(ServicePort.Type.HTTP));
     } finally {
     } finally {
       EntityUtils.consume(entity);
       EntityUtils.consume(entity);
     }
     }

+ 24 - 23
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 
 
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneAcl;
@@ -169,11 +170,11 @@ public class OzoneBucket {
    */
    */
   public void putKey(String keyName, String data) throws OzoneException {
   public void putKey(String keyName, String data) throws OzoneException {
     if ((keyName == null) || keyName.isEmpty()) {
     if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneRestClientException("Invalid key Name.");
+      throw new OzoneClientException("Invalid key Name.");
     }
     }
 
 
     if (data == null) {
     if (data == null) {
-      throw new OzoneRestClientException("Invalid data.");
+      throw new OzoneClientException("Invalid data.");
     }
     }
 
 
     HttpPut putRequest = null;
     HttpPut putRequest = null;
@@ -195,7 +196,7 @@ public class OzoneBucket {
       }
       }
       executePutKey(putRequest, httpClient);
       executePutKey(putRequest, httpClient);
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       IOUtils.closeStream(is);
       IOUtils.closeStream(is);
       releaseConnection(putRequest);
       releaseConnection(putRequest);
@@ -211,7 +212,7 @@ public class OzoneBucket {
    */
    */
   public void putKey(File dataFile) throws OzoneException {
   public void putKey(File dataFile) throws OzoneException {
     if (dataFile == null) {
     if (dataFile == null) {
-      throw new OzoneRestClientException("Invalid file object.");
+      throw new OzoneClientException("Invalid file object.");
     }
     }
     String keyName = dataFile.getName();
     String keyName = dataFile.getName();
     putKey(keyName, dataFile);
     putKey(keyName, dataFile);
@@ -228,11 +229,11 @@ public class OzoneBucket {
       throws OzoneException {
       throws OzoneException {
 
 
     if ((keyName == null) || keyName.isEmpty()) {
     if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneRestClientException("Invalid key Name");
+      throw new OzoneClientException("Invalid key Name");
     }
     }
 
 
     if (file == null) {
     if (file == null) {
-      throw new OzoneRestClientException("Invalid data stream");
+      throw new OzoneClientException("Invalid data stream");
     }
     }
 
 
     HttpPut putRequest = null;
     HttpPut putRequest = null;
@@ -253,7 +254,7 @@ public class OzoneBucket {
       executePutKey(putRequest, httpClient);
       executePutKey(putRequest, httpClient);
 
 
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      final OzoneRestClientException orce = new OzoneRestClientException(
+      final OzoneClientException orce = new OzoneClientException(
           "Failed to putKey: keyName=" + keyName + ", file=" + file);
           "Failed to putKey: keyName=" + keyName + ", file=" + file);
       orce.initCause(ex);
       orce.initCause(ex);
       LOG.trace("", orce);
       LOG.trace("", orce);
@@ -285,7 +286,7 @@ public class OzoneBucket {
       }
       }
 
 
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
 
 
       throw OzoneException.parse(EntityUtils.toString(entity));
       throw OzoneException.parse(EntityUtils.toString(entity));
@@ -306,11 +307,11 @@ public class OzoneBucket {
   public void getKey(String keyName, Path downloadTo) throws OzoneException {
   public void getKey(String keyName, Path downloadTo) throws OzoneException {
 
 
     if ((keyName == null) || keyName.isEmpty()) {
     if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneRestClientException("Invalid key Name");
+      throw new OzoneClientException("Invalid key Name");
     }
     }
 
 
     if (downloadTo == null) {
     if (downloadTo == null) {
-      throw new OzoneRestClientException("Invalid download path");
+      throw new OzoneClientException("Invalid download path");
     }
     }
 
 
     FileOutputStream outPutFile = null;
     FileOutputStream outPutFile = null;
@@ -326,7 +327,7 @@ public class OzoneBucket {
       executeGetKey(getRequest, httpClient, outPutFile);
       executeGetKey(getRequest, httpClient, outPutFile);
       outPutFile.flush();
       outPutFile.flush();
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       IOUtils.closeStream(outPutFile);
       IOUtils.closeStream(outPutFile);
       releaseConnection(getRequest);
       releaseConnection(getRequest);
@@ -343,7 +344,7 @@ public class OzoneBucket {
   public String getKey(String keyName) throws OzoneException {
   public String getKey(String keyName) throws OzoneException {
 
 
     if ((keyName == null) || keyName.isEmpty()) {
     if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneRestClientException("Invalid key Name");
+      throw new OzoneClientException("Invalid key Name");
     }
     }
 
 
     HttpGet getRequest = null;
     HttpGet getRequest = null;
@@ -360,7 +361,7 @@ public class OzoneBucket {
       executeGetKey(getRequest, httpClient, outPutStream);
       executeGetKey(getRequest, httpClient, outPutStream);
       return outPutStream.toString(ENCODING_NAME);
       return outPutStream.toString(ENCODING_NAME);
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       IOUtils.closeStream(outPutStream);
       IOUtils.closeStream(outPutStream);
       releaseConnection(getRequest);
       releaseConnection(getRequest);
@@ -394,7 +395,7 @@ public class OzoneBucket {
       }
       }
 
 
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
 
 
       throw OzoneException.parse(EntityUtils.toString(entity));
       throw OzoneException.parse(EntityUtils.toString(entity));
@@ -414,7 +415,7 @@ public class OzoneBucket {
   public void deleteKey(String keyName) throws OzoneException {
   public void deleteKey(String keyName) throws OzoneException {
 
 
     if ((keyName == null) || keyName.isEmpty()) {
     if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneRestClientException("Invalid key Name");
+      throw new OzoneClientException("Invalid key Name");
     }
     }
 
 
     HttpDelete deleteRequest = null;
     HttpDelete deleteRequest = null;
@@ -427,7 +428,7 @@ public class OzoneBucket {
           .getClient().getHttpDelete(builder.toString());
           .getClient().getHttpDelete(builder.toString());
       executeDeleteKey(deleteRequest, httpClient);
       executeDeleteKey(deleteRequest, httpClient);
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(deleteRequest);
       releaseConnection(deleteRequest);
     }
     }
@@ -457,7 +458,7 @@ public class OzoneBucket {
       }
       }
 
 
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
 
 
       throw OzoneException.parse(EntityUtils.toString(entity));
       throw OzoneException.parse(EntityUtils.toString(entity));
@@ -505,7 +506,7 @@ public class OzoneBucket {
       return executeListKeys(getRequest, httpClient);
       return executeListKeys(getRequest, httpClient);
 
 
     } catch (IOException | URISyntaxException e) {
     } catch (IOException | URISyntaxException e) {
-      throw new OzoneRestClientException(e.getMessage(), e);
+      throw new OzoneClientException(e.getMessage(), e);
     } finally {
     } finally {
       releaseConnection(getRequest);
       releaseConnection(getRequest);
     }
     }
@@ -534,7 +535,7 @@ public class OzoneBucket {
       getRequest = client.getHttpGet(uri.toString());
       getRequest = client.getHttpGet(uri.toString());
       return executeListKeys(getRequest, HttpClientBuilder.create().build());
       return executeListKeys(getRequest, HttpClientBuilder.create().build());
     } catch (IOException | URISyntaxException e) {
     } catch (IOException | URISyntaxException e) {
-      throw new OzoneRestClientException(e.getMessage());
+      throw new OzoneClientException(e.getMessage());
     } finally {
     } finally {
       releaseConnection(getRequest);
       releaseConnection(getRequest);
     }
     }
@@ -560,7 +561,7 @@ public class OzoneBucket {
       entity = response.getEntity();
       entity = response.getEntity();
 
 
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
       if (errorCode == HTTP_OK) {
       if (errorCode == HTTP_OK) {
         String temp = EntityUtils.toString(entity);
         String temp = EntityUtils.toString(entity);
@@ -586,7 +587,7 @@ public class OzoneBucket {
    */
    */
   public OzoneKey getKeyInfo(String keyName) throws OzoneException {
   public OzoneKey getKeyInfo(String keyName) throws OzoneException {
     if ((keyName == null) || keyName.isEmpty()) {
     if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Unable to get key info, key name is null or empty");
           "Unable to get key info, key name is null or empty");
     }
     }
 
 
@@ -604,7 +605,7 @@ public class OzoneBucket {
       getRequest = client.getHttpGet(builder.toString());
       getRequest = client.getHttpGet(builder.toString());
       return executeGetKeyInfo(getRequest, httpClient);
       return executeGetKeyInfo(getRequest, httpClient);
     } catch (IOException | URISyntaxException e) {
     } catch (IOException | URISyntaxException e) {
-      throw new OzoneRestClientException(e.getMessage(), e);
+      throw new OzoneClientException(e.getMessage(), e);
     } finally {
     } finally {
       releaseConnection(getRequest);
       releaseConnection(getRequest);
     }
     }
@@ -627,7 +628,7 @@ public class OzoneBucket {
       int errorCode = response.getStatusLine().getStatusCode();
       int errorCode = response.getStatusLine().getStatusCode();
       entity = response.getEntity();
       entity = response.getEntity();
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
 
 
       if (errorCode == HTTP_OK) {
       if (errorCode == HTTP_OK) {

+ 22 - 21
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java

@@ -24,6 +24,7 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.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.ListVolumes;
@@ -111,7 +112,7 @@ public class OzoneRestClient implements Closeable {
    */
    */
   public void setEndPointURI(URI endPointURI) throws OzoneException {
   public void setEndPointURI(URI endPointURI) throws OzoneException {
     if ((endPointURI == null) || (endPointURI.toString().isEmpty())) {
     if ((endPointURI == null) || (endPointURI.toString().isEmpty())) {
-      throw new OzoneRestClientException("Invalid ozone URI");
+      throw new OzoneClientException("Invalid ozone URI");
     }
     }
     this.endPointURI = endPointURI;
     this.endPointURI = endPointURI;
   }
   }
@@ -151,7 +152,7 @@ public class OzoneRestClient implements Closeable {
    * @param onBehalfOf - The user on behalf we are making the call for
    * @param onBehalfOf - The user on behalf we are making the call for
    * @param quota      - Quota's are specified in a specific format. it is
    * @param quota      - Quota's are specified in a specific format. it is
    *                   integer(MB|GB|TB), for example 100TB.
    *                   integer(MB|GB|TB), for example 100TB.
-   * @throws OzoneRestClientException
+   * @throws OzoneClientException
    */
    */
   public OzoneVolume createVolume(String volumeName, String onBehalfOf,
   public OzoneVolume createVolume(String volumeName, String onBehalfOf,
                                   String quota) throws OzoneException {
                                   String quota) throws OzoneException {
@@ -169,7 +170,7 @@ public class OzoneRestClient implements Closeable {
       executeCreateVolume(httpPost, httpClient);
       executeCreateVolume(httpPost, httpClient);
       return getVolume(volumeName);
       return getVolume(volumeName);
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(httpPost);
       releaseConnection(httpPost);
     }
     }
@@ -196,7 +197,7 @@ public class OzoneRestClient implements Closeable {
       httpGet = getHttpGet(builder.toString());
       httpGet = getHttpGet(builder.toString());
       return executeInfoVolume(httpGet, httpClient);
       return executeInfoVolume(httpGet, httpClient);
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(httpGet);
       releaseConnection(httpGet);
     }
     }
@@ -247,7 +248,7 @@ public class OzoneRestClient implements Closeable {
       }
       }
       return executeListVolume(httpGet, httpClient);
       return executeListVolume(httpGet, httpClient);
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(httpGet);
       releaseConnection(httpGet);
     }
     }
@@ -329,7 +330,7 @@ public class OzoneRestClient implements Closeable {
       return executeListVolume(httpGet, httpClient);
       return executeListVolume(httpGet, httpClient);
 
 
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(httpGet);
       releaseConnection(httpGet);
     }
     }
@@ -351,7 +352,7 @@ public class OzoneRestClient implements Closeable {
       httpDelete = getHttpDelete(builder.toString());
       httpDelete = getHttpDelete(builder.toString());
       executeDeleteVolume(httpDelete, httpClient);
       executeDeleteVolume(httpDelete, httpClient);
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(httpDelete);
       releaseConnection(httpDelete);
     }
     }
@@ -368,7 +369,7 @@ public class OzoneRestClient implements Closeable {
       throws OzoneException {
       throws OzoneException {
     HttpPut putRequest = null;
     HttpPut putRequest = null;
     if (newOwner == null || newOwner.isEmpty()) {
     if (newOwner == null || newOwner.isEmpty()) {
-      throw new OzoneRestClientException("Invalid new owner name");
+      throw new OzoneClientException("Invalid new owner name");
     }
     }
     try (CloseableHttpClient httpClient = newHttpClient()) {
     try (CloseableHttpClient httpClient = newHttpClient()) {
       OzoneUtils.verifyResourceName(volumeName);
       OzoneUtils.verifyResourceName(volumeName);
@@ -380,7 +381,7 @@ public class OzoneRestClient implements Closeable {
       executePutVolume(putRequest, httpClient);
       executePutVolume(putRequest, httpClient);
 
 
     } catch (URISyntaxException | IllegalArgumentException | IOException ex) {
     } catch (URISyntaxException | IllegalArgumentException | IOException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(putRequest);
       releaseConnection(putRequest);
     }
     }
@@ -399,7 +400,7 @@ public class OzoneRestClient implements Closeable {
   public void setVolumeQuota(String volumeName, String quota)
   public void setVolumeQuota(String volumeName, String quota)
       throws OzoneException {
       throws OzoneException {
     if (quota == null || quota.isEmpty()) {
     if (quota == null || quota.isEmpty()) {
-      throw new OzoneRestClientException("Invalid quota");
+      throw new OzoneClientException("Invalid quota");
     }
     }
     HttpPut putRequest = null;
     HttpPut putRequest = null;
     try (CloseableHttpClient httpClient = newHttpClient()) {
     try (CloseableHttpClient httpClient = newHttpClient()) {
@@ -413,7 +414,7 @@ public class OzoneRestClient implements Closeable {
       executePutVolume(putRequest, httpClient);
       executePutVolume(putRequest, httpClient);
 
 
     } catch (URISyntaxException | IllegalArgumentException | IOException ex) {
     } catch (URISyntaxException | IllegalArgumentException | IOException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(putRequest);
       releaseConnection(putRequest);
     }
     }
@@ -443,7 +444,7 @@ public class OzoneRestClient implements Closeable {
       if (entity != null) {
       if (entity != null) {
         throw OzoneException.parse(EntityUtils.toString(entity));
         throw OzoneException.parse(EntityUtils.toString(entity));
       } else {
       } else {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
     } finally {
     } finally {
       if (entity != null) {
       if (entity != null) {
@@ -470,7 +471,7 @@ public class OzoneRestClient implements Closeable {
 
 
       entity = response.getEntity();
       entity = response.getEntity();
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
 
 
       if (errorCode == HTTP_OK) {
       if (errorCode == HTTP_OK) {
@@ -531,7 +532,7 @@ public class OzoneRestClient implements Closeable {
       entity = response.getEntity();
       entity = response.getEntity();
 
 
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
 
 
       String temp = EntityUtils.toString(entity);
       String temp = EntityUtils.toString(entity);
@@ -595,11 +596,11 @@ public class OzoneRestClient implements Closeable {
     OzoneUtils.verifyResourceName(bucketName);
     OzoneUtils.verifyResourceName(bucketName);
 
 
     if (StringUtils.isEmpty(keyName)) {
     if (StringUtils.isEmpty(keyName)) {
-      throw new OzoneRestClientException("Invalid key Name");
+      throw new OzoneClientException("Invalid key Name");
     }
     }
 
 
     if (file == null) {
     if (file == null) {
-      throw new OzoneRestClientException("Invalid data stream");
+      throw new OzoneClientException("Invalid data stream");
     }
     }
 
 
     HttpPut putRequest = null;
     HttpPut putRequest = null;
@@ -619,7 +620,7 @@ public class OzoneRestClient implements Closeable {
       putRequest.setHeader(Header.CONTENT_MD5, DigestUtils.md5Hex(fis));
       putRequest.setHeader(Header.CONTENT_MD5, DigestUtils.md5Hex(fis));
       OzoneBucket.executePutKey(putRequest, httpClient);
       OzoneBucket.executePutKey(putRequest, httpClient);
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       IOUtils.closeStream(fis);
       IOUtils.closeStream(fis);
       releaseConnection(putRequest);
       releaseConnection(putRequest);
@@ -641,11 +642,11 @@ public class OzoneRestClient implements Closeable {
     OzoneUtils.verifyResourceName(bucketName);
     OzoneUtils.verifyResourceName(bucketName);
 
 
     if (StringUtils.isEmpty(keyName)) {
     if (StringUtils.isEmpty(keyName)) {
-      throw new OzoneRestClientException("Invalid key Name");
+      throw new OzoneClientException("Invalid key Name");
     }
     }
 
 
     if (downloadTo == null) {
     if (downloadTo == null) {
-      throw new OzoneRestClientException("Invalid download path");
+      throw new OzoneClientException("Invalid download path");
     }
     }
 
 
     FileOutputStream outPutFile = null;
     FileOutputStream outPutFile = null;
@@ -661,7 +662,7 @@ public class OzoneRestClient implements Closeable {
       OzoneBucket.executeGetKey(getRequest, httpClient, outPutFile);
       OzoneBucket.executeGetKey(getRequest, httpClient, outPutFile);
       outPutFile.flush();
       outPutFile.flush();
     } catch (IOException | URISyntaxException ex) {
     } catch (IOException | URISyntaxException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       IOUtils.closeStream(outPutFile);
       IOUtils.closeStream(outPutFile);
       releaseConnection(getRequest);
       releaseConnection(getRequest);
@@ -706,7 +707,7 @@ public class OzoneRestClient implements Closeable {
       getRequest = getHttpGet(builder.toString());
       getRequest = getHttpGet(builder.toString());
       return OzoneBucket.executeListKeys(getRequest, httpClient);
       return OzoneBucket.executeListKeys(getRequest, httpClient);
     } catch (IOException | URISyntaxException e) {
     } catch (IOException | URISyntaxException e) {
-      throw new OzoneRestClientException(e.getMessage(), e);
+      throw new OzoneClientException(e.getMessage(), e);
     } finally {
     } finally {
       releaseConnection(getRequest);
       releaseConnection(getRequest);
     }
     }

+ 12 - 11
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java

@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
 import com.google.common.base.Strings;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.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.request.OzoneQuota;
@@ -203,7 +204,7 @@ public class OzoneVolume {
       executeCreateBucket(httpPost, httpClient);
       executeCreateBucket(httpPost, httpClient);
       return getBucket(bucketName);
       return getBucket(bucketName);
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(httpPost);
       releaseConnection(httpPost);
     }
     }
@@ -276,7 +277,7 @@ public class OzoneVolume {
       if (entity != null) {
       if (entity != null) {
         throw OzoneException.parse(EntityUtils.toString(entity));
         throw OzoneException.parse(EntityUtils.toString(entity));
       } else {
       } else {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
     } finally {
     } finally {
       if (entity != null) {
       if (entity != null) {
@@ -307,7 +308,7 @@ public class OzoneVolume {
       }
       }
       executePutBucket(putRequest, httpClient);
       executePutBucket(putRequest, httpClient);
     } catch (URISyntaxException | IOException ex) {
     } catch (URISyntaxException | IOException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(putRequest);
       releaseConnection(putRequest);
     }
     }
@@ -336,7 +337,7 @@ public class OzoneVolume {
       }
       }
       executePutBucket(putRequest, httpClient);
       executePutBucket(putRequest, httpClient);
     } catch (URISyntaxException | IOException ex) {
     } catch (URISyntaxException | IOException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(putRequest);
       releaseConnection(putRequest);
     }
     }
@@ -361,7 +362,7 @@ public class OzoneVolume {
       return executeInfoBucket(getRequest, httpClient);
       return executeInfoBucket(getRequest, httpClient);
 
 
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(getRequest);
       releaseConnection(getRequest);
     }
     }
@@ -388,7 +389,7 @@ public class OzoneVolume {
       int errorCode = response.getStatusLine().getStatusCode();
       int errorCode = response.getStatusLine().getStatusCode();
       entity = response.getEntity();
       entity = response.getEntity();
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
       if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
       if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
         OzoneBucket bucket =
         OzoneBucket bucket =
@@ -432,7 +433,7 @@ public class OzoneVolume {
         throw OzoneException.parse(EntityUtils.toString(entity));
         throw OzoneException.parse(EntityUtils.toString(entity));
       }
       }
 
 
-      throw new OzoneRestClientException("Unexpected null in http result");
+      throw new OzoneClientException("Unexpected null in http result");
     } finally {
     } finally {
       if (entity != null) {
       if (entity != null) {
         EntityUtils.consumeQuietly(entity);
         EntityUtils.consumeQuietly(entity);
@@ -467,7 +468,7 @@ public class OzoneVolume {
       return executeListBuckets(getRequest, httpClient);
       return executeListBuckets(getRequest, httpClient);
 
 
     } catch (IOException | URISyntaxException e) {
     } catch (IOException | URISyntaxException e) {
-      throw new OzoneRestClientException(e.getMessage(), e);
+      throw new OzoneClientException(e.getMessage(), e);
     } finally {
     } finally {
       releaseConnection(getRequest);
       releaseConnection(getRequest);
     }
     }
@@ -496,7 +497,7 @@ public class OzoneVolume {
       entity = response.getEntity();
       entity = response.getEntity();
 
 
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload");
+        throw new OzoneClientException("Unexpected null in http payload");
       }
       }
       if (errorCode == HTTP_OK) {
       if (errorCode == HTTP_OK) {
         ListBuckets bucketList =
         ListBuckets bucketList =
@@ -535,7 +536,7 @@ public class OzoneVolume {
       executeDeleteBucket(delRequest, httpClient);
       executeDeleteBucket(delRequest, httpClient);
 
 
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
     } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneRestClientException(ex.getMessage(), ex);
+      throw new OzoneClientException(ex.getMessage(), ex);
     } finally {
     } finally {
       releaseConnection(delRequest);
       releaseConnection(delRequest);
     }
     }
@@ -564,7 +565,7 @@ public class OzoneVolume {
       }
       }
 
 
       if (entity == null) {
       if (entity == null) {
-        throw new OzoneRestClientException("Unexpected null in http payload.");
+        throw new OzoneClientException("Unexpected null in http payload.");
       }
       }
 
 
       throw OzoneException.parse(EntityUtils.toString(entity));
       throw OzoneException.parse(EntityUtils.toString(entity));

+ 207 - 91
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.ozone.ozShell;
 package org.apache.hadoop.ozone.ozShell;
 
 
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
@@ -26,15 +27,21 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
+import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.List;
 import java.util.List;
 import java.util.Random;
 import java.util.Random;
 import java.util.UUID;
 import java.util.UUID;
+import java.util.stream.Collectors;
 
 
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneAcl;
@@ -43,11 +50,16 @@ import org.apache.hadoop.ozone.OzoneAcl.OzoneACLType;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.web.client.OzoneBucket;
-import org.apache.hadoop.ozone.web.client.OzoneKey;
-import org.apache.hadoop.ozone.web.client.OzoneRestClient;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.rest.RestClient;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.response.BucketInfo;
@@ -63,12 +75,20 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
 /**
  * This test class specified for testing Ozone shell command.
  * This test class specified for testing Ozone shell command.
  */
  */
+@RunWith(value = Parameterized.class)
 public class TestOzoneShell {
 public class TestOzoneShell {
 
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOzoneShell.class);
+
   /**
   /**
    * Set the timeout for every test.
    * Set the timeout for every test.
    */
    */
@@ -79,7 +99,7 @@ public class TestOzoneShell {
   private static File baseDir;
   private static File baseDir;
   private static OzoneConfiguration conf = null;
   private static OzoneConfiguration conf = null;
   private static MiniOzoneCluster cluster = null;
   private static MiniOzoneCluster cluster = null;
-  private static OzoneRestClient client = null;
+  private static ClientProtocol client = null;
   private static Shell shell = null;
   private static Shell shell = null;
 
 
   private final ByteArrayOutputStream out = new ByteArrayOutputStream();
   private final ByteArrayOutputStream out = new ByteArrayOutputStream();
@@ -87,6 +107,16 @@ public class TestOzoneShell {
   private static final PrintStream OLD_OUT = System.out;
   private static final PrintStream OLD_OUT = System.out;
   private static final PrintStream OLD_ERR = System.err;
   private static final PrintStream OLD_ERR = System.err;
 
 
+  @Parameterized.Parameters
+  public static Collection<Object[]> clientProtocol() {
+    Object[][] params = new Object[][] {
+        {RpcClient.class},
+        {RestClient.class}};
+    return Arrays.asList(params);
+  }
+
+  @Parameterized.Parameter
+  public Class clientProtocol;
   /**
   /**
    * Create a MiniDFSCluster for testing with using distributed Ozone
    * Create a MiniDFSCluster for testing with using distributed Ozone
    * handler type.
    * handler type.
@@ -110,13 +140,12 @@ public class TestOzoneShell {
     shell = new Shell();
     shell = new Shell();
     shell.setConf(conf);
     shell.setConf(conf);
 
 
-    cluster = MiniOzoneCluster.newBuilder(conf).build();
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(3)
+        .build();
+    conf.setInt(OZONE_REPLICATION, ReplicationFactor.THREE.getValue());
+    client = new RpcClient(conf);
     cluster.waitForClusterToBeReady();
     cluster.waitForClusterToBeReady();
-    final int port = cluster.getHddsDatanodes().get(0).getDatanodeDetails()
-        .getOzoneRestPort();
-    url = String.format("http://localhost:%d", port);
-    client = new OzoneRestClient(String.format("http://localhost:%d", port));
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
   }
   }
 
 
   /**
   /**
@@ -137,6 +166,26 @@ public class TestOzoneShell {
   public void setup() {
   public void setup() {
     System.setOut(new PrintStream(out));
     System.setOut(new PrintStream(out));
     System.setErr(new PrintStream(err));
     System.setErr(new PrintStream(err));
+    if(clientProtocol.equals(RestClient.class)) {
+      String hostName = cluster.getKeySpaceManager().getHttpServer()
+          .getHttpAddress().getHostName();
+      int port = cluster
+          .getKeySpaceManager().getHttpServer().getHttpAddress().getPort();
+      url = String.format("http://" + hostName + ":" + port);
+    } else {
+      List<ServiceInfo> services = null;
+      try {
+        services = cluster.getKeySpaceManager().getServiceList();
+      } catch (IOException e) {
+        LOG.error("Could not get service list from KSM");
+      }
+      String hostName = services.stream().filter(
+          a -> a.getNodeType().equals(HddsProtos.NodeType.KSM))
+          .collect(Collectors.toList()).get(0).getHostname();
+
+      String port = cluster.getKeySpaceManager().getRpcPort();
+      url = String.format("o3://" + hostName + ":" + port);
+    }
   }
   }
 
 
   @After
   @After
@@ -152,22 +201,29 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testCreateVolume() throws Exception {
   public void testCreateVolume() throws Exception {
+    LOG.info("Running testCreateVolume");
     String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
     String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
     String userName = "bilbo";
     String userName = "bilbo";
     String[] args = new String[] {"-createVolume", url + "/" + volumeName,
     String[] args = new String[] {"-createVolume", url + "/" + volumeName,
         "-user", userName, "-root"};
         "-user", userName, "-root"};
 
 
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
-    OzoneVolume volumeInfo = client.getVolume(volumeName);
-    assertEquals(volumeName, volumeInfo.getVolumeName());
-    assertEquals(userName, volumeInfo.getOwnerName());
+    OzoneVolume volumeInfo = client.getVolumeDetails(volumeName);
+    assertEquals(volumeName, volumeInfo.getName());
+    assertEquals(userName, volumeInfo.getOwner());
   }
   }
 
 
   @Test
   @Test
   public void testDeleteVolume() throws Exception {
   public void testDeleteVolume() throws Exception {
+    LOG.info("Running testDeleteVolume");
     String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
     String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(vol);
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume volume = client.getVolumeDetails(volumeName);
+    assertNotNull(volume);
 
 
     String[] args = new String[] {"-deleteVolume", url + "/" + volumeName,
     String[] args = new String[] {"-deleteVolume", url + "/" + volumeName,
         "-root"};
         "-root"};
@@ -175,9 +231,9 @@ public class TestOzoneShell {
 
 
     // verify if volume has been deleted
     // verify if volume has been deleted
     try {
     try {
-      client.getVolume(volumeName);
+      client.getVolumeDetails(volumeName);
       fail("Get volume call should have thrown.");
       fail("Get volume call should have thrown.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
       GenericTestUtils.assertExceptionContains(
           "Info Volume failed, error:VOLUME_NOT_FOUND", e);
           "Info Volume failed, error:VOLUME_NOT_FOUND", e);
     }
     }
@@ -185,8 +241,13 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testInfoVolume() throws Exception {
   public void testInfoVolume() throws Exception {
+    LOG.info("Running testInfoVolume");
     String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
     String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
-    client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
 
 
     String[] args = new String[] {"-infoVolume", url + "/" + volumeName,
     String[] args = new String[] {"-infoVolume", url + "/" + volumeName,
         "-root"};
         "-root"};
@@ -206,45 +267,53 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testUpdateVolume() throws Exception {
   public void testUpdateVolume() throws Exception {
+    LOG.info("Running testUpdateVolume");
     String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
     String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
     String userName = "bilbo";
     String userName = "bilbo";
-    OzoneVolume vol = client.createVolume(volumeName, userName, "100TB");
-    assertEquals(userName, vol.getOwnerName());
-    assertEquals(100, vol.getQuota().getSize(), 100);
-    assertEquals(OzoneQuota.Units.TB, vol.getQuota().getUnit());
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
+    assertEquals(userName, vol.getOwner());
+    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(), vol.getQuota());
 
 
     String[] args = new String[] {"-updateVolume", url + "/" + volumeName,
     String[] args = new String[] {"-updateVolume", url + "/" + volumeName,
         "-quota", "500MB", "-root"};
         "-quota", "500MB", "-root"};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
-    vol = client.getVolume(volumeName);
-    assertEquals(userName, vol.getOwnerName());
-    assertEquals(500, vol.getQuota().getSize(), 500);
-    assertEquals(OzoneQuota.Units.MB, vol.getQuota().getUnit());
+    vol = client.getVolumeDetails(volumeName);
+    assertEquals(userName, vol.getOwner());
+    assertEquals(OzoneQuota.parseQuota("500MB").sizeInBytes(), vol.getQuota());
 
 
     String newUser = "new-user";
     String newUser = "new-user";
     args = new String[] {"-updateVolume", url + "/" + volumeName,
     args = new String[] {"-updateVolume", url + "/" + volumeName,
         "-user", newUser, "-root"};
         "-user", newUser, "-root"};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
-    vol = client.getVolume(volumeName);
-    assertEquals(newUser, vol.getOwnerName());
+    vol = client.getVolumeDetails(volumeName);
+    assertEquals(newUser, vol.getOwner());
 
 
     // test error conditions
     // test error conditions
     args = new String[] {"-updateVolume", url + "/invalid-volume",
     args = new String[] {"-updateVolume", url + "/invalid-volume",
         "-user", newUser, "-root"};
         "-user", newUser, "-root"};
     assertEquals(1, ToolRunner.run(shell, args));
     assertEquals(1, ToolRunner.run(shell, args));
     assertTrue(err.toString().contains(
     assertTrue(err.toString().contains(
-        "Volume owner change failed, error:VOLUME_NOT_FOUND"));
+        "Info Volume failed, error:VOLUME_NOT_FOUND"));
 
 
     err.reset();
     err.reset();
     args = new String[] {"-updateVolume", url + "/invalid-volume",
     args = new String[] {"-updateVolume", url + "/invalid-volume",
         "-quota", "500MB", "-root"};
         "-quota", "500MB", "-root"};
     assertEquals(1, ToolRunner.run(shell, args));
     assertEquals(1, ToolRunner.run(shell, args));
     assertTrue(err.toString().contains(
     assertTrue(err.toString().contains(
-        "Volume quota change failed, error:VOLUME_NOT_FOUND"));
+        "Info Volume failed, error:VOLUME_NOT_FOUND"));
   }
   }
 
 
   @Test
   @Test
   public void testListVolume() throws Exception {
   public void testListVolume() throws Exception {
+    LOG.info("Running testListVolume");
+    if (clientProtocol.equals(RestClient.class)) {
+      return;
+    }
     String commandOutput;
     String commandOutput;
     List<VolumeInfo> volumes;
     List<VolumeInfo> volumes;
     final int volCount = 20;
     final int volCount = 20;
@@ -265,7 +334,12 @@ public class TestOzoneShell {
         userName = user2;
         userName = user2;
         volumeName = "test-vol" + x;
         volumeName = "test-vol" + x;
       }
       }
-      OzoneVolume vol = client.createVolume(volumeName, userName, "100TB");
+      VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+          .setOwner(userName)
+          .setQuota("100TB")
+          .build();
+      client.createVolume(volumeName, volumeArgs);
+      OzoneVolume vol = client.getVolumeDetails(volumeName);
       assertNotNull(vol);
       assertNotNull(vol);
     }
     }
 
 
@@ -343,16 +417,17 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testCreateBucket() throws Exception {
   public void testCreateBucket() throws Exception {
+    LOG.info("Running testCreateBucket");
     OzoneVolume vol = creatVolume();
     OzoneVolume vol = creatVolume();
     String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
     String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
     String[] args = new String[] {"-createBucket",
     String[] args = new String[] {"-createBucket",
-        url + "/" + vol.getVolumeName() + "/" + bucketName};
+        url + "/" + vol.getName() + "/" + bucketName};
 
 
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
     OzoneBucket bucketInfo = vol.getBucket(bucketName);
     OzoneBucket bucketInfo = vol.getBucket(bucketName);
-    assertEquals(vol.getVolumeName(),
-        bucketInfo.getBucketInfo().getVolumeName());
-    assertEquals(bucketName, bucketInfo.getBucketName());
+    assertEquals(vol.getName(),
+        bucketInfo.getVolumeName());
+    assertEquals(bucketName, bucketInfo.getName());
 
 
     // test create a bucket in a non-exist volume
     // test create a bucket in a non-exist volume
     args = new String[] {"-createBucket",
     args = new String[] {"-createBucket",
@@ -365,20 +440,22 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testDeleteBucket() throws Exception {
   public void testDeleteBucket() throws Exception {
+    LOG.info("Running testDeleteBucket");
     OzoneVolume vol = creatVolume();
     OzoneVolume vol = creatVolume();
     String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
     String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
-    OzoneBucket bucketInfo = vol.createBucket(bucketName);
+    vol.createBucket(bucketName);
+    OzoneBucket bucketInfo = vol.getBucket(bucketName);
     assertNotNull(bucketInfo);
     assertNotNull(bucketInfo);
 
 
     String[] args = new String[] {"-deleteBucket",
     String[] args = new String[] {"-deleteBucket",
-        url + "/" + vol.getVolumeName() + "/" + bucketName};
+        url + "/" + vol.getName() + "/" + bucketName};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
 
 
     // verify if bucket has been deleted in volume
     // verify if bucket has been deleted in volume
     try {
     try {
       vol.getBucket(bucketName);
       vol.getBucket(bucketName);
       fail("Get bucket should have thrown.");
       fail("Get bucket should have thrown.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
       GenericTestUtils.assertExceptionContains(
           "Info Bucket failed, error: BUCKET_NOT_FOUND", e);
           "Info Bucket failed, error: BUCKET_NOT_FOUND", e);
     }
     }
@@ -393,7 +470,7 @@ public class TestOzoneShell {
     err.reset();
     err.reset();
     // test delete non-exist bucket
     // test delete non-exist bucket
     args = new String[] {"-deleteBucket",
     args = new String[] {"-deleteBucket",
-        url + "/" + vol.getVolumeName() + "/invalid-bucket"};
+        url + "/" + vol.getName() + "/invalid-bucket"};
     assertEquals(1, ToolRunner.run(shell, args));
     assertEquals(1, ToolRunner.run(shell, args));
     assertTrue(err.toString().contains(
     assertTrue(err.toString().contains(
         "Delete Bucket failed, error:BUCKET_NOT_FOUND"));
         "Delete Bucket failed, error:BUCKET_NOT_FOUND"));
@@ -401,12 +478,13 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testInfoBucket() throws Exception {
   public void testInfoBucket() throws Exception {
+    LOG.info("Running testInfoBucket");
     OzoneVolume vol = creatVolume();
     OzoneVolume vol = creatVolume();
     String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
     String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
     vol.createBucket(bucketName);
     vol.createBucket(bucketName);
 
 
     String[] args = new String[] {"-infoBucket",
     String[] args = new String[] {"-infoBucket",
-        url + "/" + vol.getVolumeName() + "/" + bucketName};
+        url + "/" + vol.getName() + "/" + bucketName};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
 
 
     String output = out.toString();
     String output = out.toString();
@@ -416,7 +494,7 @@ public class TestOzoneShell {
 
 
     // test get info from a non-exist bucket
     // test get info from a non-exist bucket
     args = new String[] {"-infoBucket",
     args = new String[] {"-infoBucket",
-        url + "/" + vol.getVolumeName() + "/invalid-bucket" + bucketName};
+        url + "/" + vol.getName() + "/invalid-bucket" + bucketName};
     assertEquals(1, ToolRunner.run(shell, args));
     assertEquals(1, ToolRunner.run(shell, args));
     assertTrue(err.toString().contains(
     assertTrue(err.toString().contains(
         "Info Bucket failed, error: BUCKET_NOT_FOUND"));
         "Info Bucket failed, error: BUCKET_NOT_FOUND"));
@@ -424,13 +502,15 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testUpdateBucket() throws Exception {
   public void testUpdateBucket() throws Exception {
+    LOG.info("Running testUpdateBucket");
     OzoneVolume vol = creatVolume();
     OzoneVolume vol = creatVolume();
     String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
     String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
-    OzoneBucket bucket = vol.createBucket(bucketName);
-    assertEquals(0, bucket.getAcls().size());
+    vol.createBucket(bucketName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    int aclSize = bucket.getAcls().size();
 
 
     String[] args = new String[] {"-updateBucket",
     String[] args = new String[] {"-updateBucket",
-        url + "/" + vol.getVolumeName() + "/" + bucketName, "-addAcl",
+        url + "/" + vol.getName() + "/" + bucketName, "-addAcl",
         "user:frodo:rw,group:samwise:r"};
         "user:frodo:rw,group:samwise:r"};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
     String output = out.toString();
     String output = out.toString();
@@ -438,36 +518,40 @@ public class TestOzoneShell {
         && output.contains(OzoneConsts.OZONE_TIME_ZONE));
         && output.contains(OzoneConsts.OZONE_TIME_ZONE));
 
 
     bucket = vol.getBucket(bucketName);
     bucket = vol.getBucket(bucketName);
-    assertEquals(2, bucket.getAcls().size());
+    assertEquals(2 + aclSize, bucket.getAcls().size());
 
 
-    OzoneAcl acl = bucket.getAcls().get(0);
+    OzoneAcl acl = bucket.getAcls().get(aclSize);
     assertTrue(acl.getName().equals("frodo")
     assertTrue(acl.getName().equals("frodo")
         && acl.getType() == OzoneACLType.USER
         && acl.getType() == OzoneACLType.USER
         && acl.getRights()== OzoneACLRights.READ_WRITE);
         && acl.getRights()== OzoneACLRights.READ_WRITE);
 
 
     args = new String[] {"-updateBucket",
     args = new String[] {"-updateBucket",
-        url + "/" + vol.getVolumeName() + "/" + bucketName, "-removeAcl",
+        url + "/" + vol.getName() + "/" + bucketName, "-removeAcl",
         "user:frodo:rw"};
         "user:frodo:rw"};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
 
 
     bucket = vol.getBucket(bucketName);
     bucket = vol.getBucket(bucketName);
-    acl = bucket.getAcls().get(0);
-    assertEquals(1, bucket.getAcls().size());
+    acl = bucket.getAcls().get(aclSize);
+    assertEquals(1 + aclSize, bucket.getAcls().size());
     assertTrue(acl.getName().equals("samwise")
     assertTrue(acl.getName().equals("samwise")
         && acl.getType() == OzoneACLType.GROUP
         && acl.getType() == OzoneACLType.GROUP
         && acl.getRights()== OzoneACLRights.READ);
         && acl.getRights()== OzoneACLRights.READ);
 
 
     // test update bucket for a non-exist bucket
     // test update bucket for a non-exist bucket
     args = new String[] {"-updateBucket",
     args = new String[] {"-updateBucket",
-        url + "/" + vol.getVolumeName() + "/invalid-bucket", "-addAcl",
+        url + "/" + vol.getName() + "/invalid-bucket", "-addAcl",
         "user:frodo:rw"};
         "user:frodo:rw"};
     assertEquals(1, ToolRunner.run(shell, args));
     assertEquals(1, ToolRunner.run(shell, args));
     assertTrue(err.toString().contains(
     assertTrue(err.toString().contains(
-        "Setting bucket property failed, error: BUCKET_NOT_FOUND"));
+        "Info Bucket failed, error: BUCKET_NOT_FOUND"));
   }
   }
 
 
   @Test
   @Test
   public void testListBucket() throws Exception {
   public void testListBucket() throws Exception {
+    LOG.info("Running testListBucket");
+    if (clientProtocol.equals(RestClient.class)) {
+      return;
+    }
     List<BucketInfo> buckets;
     List<BucketInfo> buckets;
     String commandOutput;
     String commandOutput;
     int bucketCount = 11;
     int bucketCount = 11;
@@ -478,13 +562,14 @@ public class TestOzoneShell {
     for (int i = 0; i < bucketCount; i++) {
     for (int i = 0; i < bucketCount; i++) {
       String name = "test-bucket" + i;
       String name = "test-bucket" + i;
       bucketNames.add(name);
       bucketNames.add(name);
-      OzoneBucket bucket = vol.createBucket(name);
+      vol.createBucket(name);
+      OzoneBucket bucket = vol.getBucket(name);
       assertNotNull(bucket);
       assertNotNull(bucket);
     }
     }
 
 
     // test -length option
     // test -length option
     String[] args = new String[] {"-listBucket",
     String[] args = new String[] {"-listBucket",
-        url + "/" + vol.getVolumeName(), "-length", "100"};
+        url + "/" + vol.getName(), "-length", "100"};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
     commandOutput = out.toString();
     commandOutput = out.toString();
     buckets = (List<BucketInfo>) JsonUtils.toJsonList(commandOutput,
     buckets = (List<BucketInfo>) JsonUtils.toJsonList(commandOutput,
@@ -497,13 +582,13 @@ public class TestOzoneShell {
     // test-bucket10, test-bucket2, ,..., test-bucket9]
     // test-bucket10, test-bucket2, ,..., test-bucket9]
     for (int i = 0; i < buckets.size(); i++) {
     for (int i = 0; i < buckets.size(); i++) {
       assertEquals(buckets.get(i).getBucketName(), bucketNames.get(i));
       assertEquals(buckets.get(i).getBucketName(), bucketNames.get(i));
-      assertEquals(buckets.get(i).getVolumeName(), vol.getVolumeName());
+      assertEquals(buckets.get(i).getVolumeName(), vol.getName());
       assertTrue(buckets.get(i).getCreatedOn()
       assertTrue(buckets.get(i).getCreatedOn()
           .contains(OzoneConsts.OZONE_TIME_ZONE));
           .contains(OzoneConsts.OZONE_TIME_ZONE));
     }
     }
 
 
     out.reset();
     out.reset();
-    args = new String[] {"-listBucket", url + "/" + vol.getVolumeName(),
+    args = new String[] {"-listBucket", url + "/" + vol.getName(),
         "-length", "3"};
         "-length", "3"};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
     commandOutput = out.toString();
     commandOutput = out.toString();
@@ -519,7 +604,7 @@ public class TestOzoneShell {
 
 
     // test -prefix option
     // test -prefix option
     out.reset();
     out.reset();
-    args = new String[] {"-listBucket", url + "/" + vol.getVolumeName(),
+    args = new String[] {"-listBucket", url + "/" + vol.getName(),
         "-length", "100", "-prefix", "test-bucket1"};
         "-length", "100", "-prefix", "test-bucket1"};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
     commandOutput = out.toString();
     commandOutput = out.toString();
@@ -533,7 +618,7 @@ public class TestOzoneShell {
 
 
     // test -start option
     // test -start option
     out.reset();
     out.reset();
-    args = new String[] {"-listBucket", url + "/" + vol.getVolumeName(),
+    args = new String[] {"-listBucket", url + "/" + vol.getName(),
         "-length", "100", "-start", "test-bucket7"};
         "-length", "100", "-start", "test-bucket7"};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
     commandOutput = out.toString();
     commandOutput = out.toString();
@@ -546,7 +631,7 @@ public class TestOzoneShell {
 
 
     // test error conditions
     // test error conditions
     err.reset();
     err.reset();
-    args = new String[] {"-listBucket", url + "/" + vol.getVolumeName(),
+    args = new String[] {"-listBucket", url + "/" + vol.getName(),
         "-length", "-1"};
         "-length", "-1"};
     assertEquals(1, ToolRunner.run(shell, args));
     assertEquals(1, ToolRunner.run(shell, args));
     assertTrue(err.toString().contains(
     assertTrue(err.toString().contains(
@@ -555,9 +640,10 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testPutKey() throws Exception {
   public void testPutKey() throws Exception {
+    LOG.info("Running testPutKey");
     OzoneBucket bucket = creatBucket();
     OzoneBucket bucket = creatBucket();
-    String volumeName = bucket.getBucketInfo().getVolumeName();
-    String bucketName = bucket.getBucketName();
+    String volumeName = bucket.getVolumeName();
+    String bucketName = bucket.getName();
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
 
 
     String[] args = new String[] {"-putKey",
     String[] args = new String[] {"-putKey",
@@ -565,8 +651,8 @@ public class TestOzoneShell {
         createTmpFile()};
         createTmpFile()};
     assertEquals(0, ToolRunner.run(shell, args));
     assertEquals(0, ToolRunner.run(shell, args));
 
 
-    OzoneKey keyInfo = bucket.getKeyInfo(keyName);
-    assertEquals(keyName, keyInfo.getObjectInfo().getKeyName());
+    OzoneKey keyInfo = bucket.getKey(keyName);
+    assertEquals(keyName, keyInfo.getName());
 
 
     // test put key in a non-exist bucket
     // test put key in a non-exist bucket
     args = new String[] {"-putKey",
     args = new String[] {"-putKey",
@@ -574,18 +660,22 @@ public class TestOzoneShell {
         createTmpFile()};
         createTmpFile()};
     assertEquals(1, ToolRunner.run(shell, args));
     assertEquals(1, ToolRunner.run(shell, args));
     assertTrue(err.toString().contains(
     assertTrue(err.toString().contains(
-        "Create key failed, error:BUCKET_NOT_FOUND"));
+        "Info Bucket failed, error: BUCKET_NOT_FOUND"));
   }
   }
 
 
   @Test
   @Test
   public void testGetKey() throws Exception {
   public void testGetKey() throws Exception {
+    LOG.info("Running testGetKey");
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
     OzoneBucket bucket = creatBucket();
     OzoneBucket bucket = creatBucket();
-    String volumeName = bucket.getBucketInfo().getVolumeName();
-    String bucketName = bucket.getBucketName();
+    String volumeName = bucket.getVolumeName();
+    String bucketName = bucket.getName();
 
 
     String dataStr = "test-data";
     String dataStr = "test-data";
-    bucket.putKey(keyName, dataStr);
+    OzoneOutputStream keyOutputStream =
+        bucket.createKey(keyName, dataStr.length());
+    keyOutputStream.write(dataStr.getBytes());
+    keyOutputStream.close();
 
 
     String tmpPath = baseDir.getAbsolutePath() + "/testfile-"
     String tmpPath = baseDir.getAbsolutePath() + "/testfile-"
         + UUID.randomUUID().toString();
         + UUID.randomUUID().toString();
@@ -603,14 +693,19 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testDeleteKey() throws Exception {
   public void testDeleteKey() throws Exception {
+    LOG.info("Running testDeleteKey");
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
     OzoneBucket bucket = creatBucket();
     OzoneBucket bucket = creatBucket();
-    String volumeName = bucket.getBucketInfo().getVolumeName();
-    String bucketName = bucket.getBucketName();
-    bucket.putKey(keyName, "test-data");
+    String volumeName = bucket.getVolumeName();
+    String bucketName = bucket.getName();
+    String dataStr = "test-data";
+    OzoneOutputStream keyOutputStream =
+        bucket.createKey(keyName, dataStr.length());
+    keyOutputStream.write(dataStr.getBytes());
+    keyOutputStream.close();
 
 
-    OzoneKey keyInfo = bucket.getKeyInfo(keyName);
-    assertEquals(keyName, keyInfo.getObjectInfo().getKeyName());
+    OzoneKey keyInfo = bucket.getKey(keyName);
+    assertEquals(keyName, keyInfo.getName());
 
 
     String[] args = new String[] {"-deleteKey",
     String[] args = new String[] {"-deleteKey",
         url + "/" + volumeName + "/" + bucketName + "/" + keyName};
         url + "/" + volumeName + "/" + bucketName + "/" + keyName};
@@ -618,9 +713,9 @@ public class TestOzoneShell {
 
 
     // verify if key has been deleted in the bucket
     // verify if key has been deleted in the bucket
     try {
     try {
-      bucket.getKeyInfo(keyName);
+      bucket.getKey(keyName);
       fail("Get key should have thrown.");
       fail("Get key should have thrown.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
       GenericTestUtils.assertExceptionContains(
           "Lookup key failed, error:KEY_NOT_FOUND", e);
           "Lookup key failed, error:KEY_NOT_FOUND", e);
     }
     }
@@ -643,22 +738,29 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testInfoKey() throws Exception {
   public void testInfoKey() throws Exception {
+    LOG.info("Running testInfoKey");
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
     OzoneBucket bucket = creatBucket();
     OzoneBucket bucket = creatBucket();
-    String volumeName = bucket.getBucketInfo().getVolumeName();
-    String bucketName = bucket.getBucketName();
-    bucket.putKey(keyName, "test-data");
+    String volumeName = bucket.getVolumeName();
+    String bucketName = bucket.getName();
+    String dataStr = "test-data";
+    OzoneOutputStream keyOutputStream =
+        bucket.createKey(keyName, dataStr.length());
+    keyOutputStream.write(dataStr.getBytes());
+    keyOutputStream.close();
 
 
     String[] args = new String[] {"-infoKey",
     String[] args = new String[] {"-infoKey",
         url + "/" + volumeName + "/" + bucketName + "/" + keyName};
         url + "/" + volumeName + "/" + bucketName + "/" + keyName};
 
 
     // verify the response output
     // verify the response output
-    assertEquals(0, ToolRunner.run(shell, args));
-
+    int a = ToolRunner.run(shell, args);
     String output = out.toString();
     String output = out.toString();
+    assertEquals(0, a);
+
     assertTrue(output.contains(keyName));
     assertTrue(output.contains(keyName));
-    assertTrue(output.contains("createdOn") && output.contains("modifiedOn")
-        && output.contains(OzoneConsts.OZONE_TIME_ZONE));
+    assertTrue(
+        output.contains("createdOn") && output.contains("modifiedOn") && output
+            .contains(OzoneConsts.OZONE_TIME_ZONE));
 
 
     // reset stream
     // reset stream
     out.reset();
     out.reset();
@@ -677,19 +779,27 @@ public class TestOzoneShell {
 
 
   @Test
   @Test
   public void testListKey() throws Exception {
   public void testListKey() throws Exception {
+    LOG.info("Running testListKey");
+    if (clientProtocol.equals(RestClient.class)) {
+      return;
+    }
     String commandOutput;
     String commandOutput;
     List<KeyInfo> keys;
     List<KeyInfo> keys;
     int keyCount = 11;
     int keyCount = 11;
     OzoneBucket bucket = creatBucket();
     OzoneBucket bucket = creatBucket();
-    String volumeName = bucket.getBucketInfo().getVolumeName();
-    String bucketName = bucket.getBucketName();
+    String volumeName = bucket.getVolumeName();
+    String bucketName = bucket.getName();
 
 
     String keyName;
     String keyName;
     List<String> keyNames = new ArrayList<>();
     List<String> keyNames = new ArrayList<>();
     for (int i = 0; i < keyCount; i++) {
     for (int i = 0; i < keyCount; i++) {
       keyName = "test-key" + i;
       keyName = "test-key" + i;
       keyNames.add(keyName);
       keyNames.add(keyName);
-      bucket.putKey(keyName, "test-data" + i);
+      String dataStr = "test-data";
+      OzoneOutputStream keyOutputStream =
+          bucket.createKey(keyName, dataStr.length());
+      keyOutputStream.write(dataStr.getBytes());
+      keyOutputStream.close();
     }
     }
 
 
     // test -length option
     // test -length option
@@ -763,17 +873,23 @@ public class TestOzoneShell {
         "the vaule should be a positive number"));
         "the vaule should be a positive number"));
   }
   }
 
 
-  private OzoneVolume creatVolume() throws OzoneException {
-    String volumeName = UUID.randomUUID().toString() + "volume";
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+  private OzoneVolume creatVolume() throws OzoneException, IOException {
+    String volumeName = RandomStringUtils.randomNumeric(5) + "volume";
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume volume = client.getVolumeDetails(volumeName);
 
 
-    return vol;
+    return volume;
   }
   }
 
 
-  private OzoneBucket creatBucket() throws OzoneException {
+  private OzoneBucket creatBucket() throws OzoneException, IOException {
     OzoneVolume vol = creatVolume();
     OzoneVolume vol = creatVolume();
-    String bucketName = UUID.randomUUID().toString() + "bucket";
-    OzoneBucket bucketInfo = vol.createBucket(bucketName);
+    String bucketName = RandomStringUtils.randomNumeric(5) + "bucket";
+    vol.createBucket(bucketName);
+    OzoneBucket bucketInfo = vol.getBucket(bucketName);
 
 
     return bucketInfo;
     return bucketInfo;
   }
   }

+ 2 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
@@ -132,7 +133,7 @@ public class TestBuckets {
       fail("Except the bucket creation to be failed because the"
       fail("Except the bucket creation to be failed because the"
           + " bucket name starts with an invalid char #");
           + " bucket name starts with an invalid char #");
     } catch (Exception e) {
     } catch (Exception e) {
-      assertTrue(e instanceof OzoneRestClientException);
+      assertTrue(e instanceof OzoneClientException);
       assertTrue(e.getMessage().contains("Bucket or Volume name"
       assertTrue(e.getMessage().contains("Bucket or Volume name"
           + " has an unsupported character : #"));
           + " has an unsupported character : #"));
     }
     }

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

@@ -25,6 +25,7 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.Status;
 import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.Status;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
@@ -141,7 +142,7 @@ public class TestVolume {
       fail("Except the volume creation be failed because the"
       fail("Except the volume creation be failed because the"
           + " volume name starts with an invalid char #");
           + " volume name starts with an invalid char #");
     } catch (Exception e) {
     } catch (Exception e) {
-      assertTrue(e instanceof OzoneRestClientException);
+      assertTrue(e instanceof OzoneClientException);
       assertTrue(e.getMessage().contains("Bucket or Volume name"
       assertTrue(e.getMessage().contains("Bucket or Volume name"
           + " has an unsupported character : #"));
           + " has an unsupported character : #"));
     }
     }

+ 43 - 16
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java

@@ -19,28 +19,30 @@
 package org.apache.hadoop.ozone.web.ozShell;
 package org.apache.hadoop.ozone.web.ozShell;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneRestClient;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.http.client.utils.URIBuilder;
 import org.apache.http.client.utils.URIBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 
 
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_HTTP_SCHEME;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+
 /**
 /**
  * Common interface for command handling.
  * Common interface for command handling.
  */
  */
 public abstract class Handler {
 public abstract class Handler {
 
 
-  protected OzoneRestClient client;
-
-  /**
-   * Constructs a client object.
-   */
-  public Handler() {
-    client = new OzoneRestClient();
-  }
+  protected static final Logger LOG = LoggerFactory.getLogger(Handler.class);
+  protected OzoneClient client;
 
 
   /**
   /**
    * Executes the Client command.
    * Executes the Client command.
@@ -61,19 +63,44 @@ public abstract class Handler {
    * @throws URISyntaxException
    * @throws URISyntaxException
    * @throws OzoneException
    * @throws OzoneException
    */
    */
-  protected URI verifyURI(String uri) throws URISyntaxException,
-      OzoneException {
+  protected URI verifyURI(String uri)
+      throws URISyntaxException, OzoneException, IOException {
     if ((uri == null) || uri.isEmpty()) {
     if ((uri == null) || uri.isEmpty()) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Ozone URI is needed to execute this command.");
           "Ozone URI is needed to execute this command.");
     }
     }
     URIBuilder ozoneURI = new URIBuilder(uri);
     URIBuilder ozoneURI = new URIBuilder(uri);
-
     if (ozoneURI.getPort() == 0) {
     if (ozoneURI.getPort() == 0) {
       ozoneURI.setPort(Shell.DEFAULT_OZONE_PORT);
       ozoneURI.setPort(Shell.DEFAULT_OZONE_PORT);
     }
     }
+
+    Configuration conf = new OzoneConfiguration();
+    String scheme = ozoneURI.getScheme();
+    if (scheme.equals(OZONE_HTTP_SCHEME)) {
+      if (ozoneURI.getHost() != null) {
+        if (ozoneURI.getPort() == -1) {
+          client = OzoneClientFactory.getRestClient(ozoneURI.getHost());
+        } else {
+          client = OzoneClientFactory
+              .getRestClient(ozoneURI.getHost(), ozoneURI.getPort(), conf);
+        }
+      } else {
+        client = OzoneClientFactory.getRestClient(conf);
+      }
+    } else if (scheme.equals(OZONE_URI_SCHEME) || scheme.isEmpty()) {
+      if (ozoneURI.getHost() != null) {
+        if (ozoneURI.getPort() == -1) {
+          client = OzoneClientFactory.getRpcClient(ozoneURI.getHost());
+        } else {
+          client = OzoneClientFactory
+              .getRpcClient(ozoneURI.getHost(), ozoneURI.getPort(), conf);
+        }
+      } else {
+        client = OzoneClientFactory.getRpcClient(conf);
+      }
+    } else {
+      throw new OzoneClientException("Invalid URI: " + ozoneURI);
+    }
     return ozoneURI.build();
     return ozoneURI.build();
   }
   }
-
-
 }
 }

+ 11 - 21
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java

@@ -18,9 +18,10 @@
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
 
 import org.apache.commons.cli.CommandLine;
 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.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -39,7 +40,6 @@ public class CreateBucketHandler extends Handler {
 
 
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
-  private String rootName;
 
 
   /**
   /**
    * Executes create bucket.
    * Executes create bucket.
@@ -54,7 +54,7 @@ public class CreateBucketHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.CREATE_BUCKET)) {
     if (!cmd.hasOption(Shell.CREATE_BUCKET)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : createBucket is missing");
           "Incorrect call : createBucket is missing");
     }
     }
 
 
@@ -62,7 +62,7 @@ public class CreateBucketHandler extends Handler {
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 2) {
     if (path.getNameCount() < 2) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume and bucket name required in createBucket");
           "volume and bucket name required in createBucket");
     }
     }
 
 
@@ -74,23 +74,13 @@ public class CreateBucketHandler extends Handler {
       System.out.printf("Bucket Name : %s%n", bucketName);
       System.out.printf("Bucket Name : %s%n", bucketName);
     }
     }
 
 
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    } else {
-      rootName = System.getProperty("user.name");
-    }
-
-
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
-
-
-    OzoneVolume vol = client.getVolume(volumeName);
-    OzoneBucket bucket = vol.createBucket(bucketName);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
+    vol.createBucket(bucketName);
 
 
     if (cmd.hasOption(Shell.VERBOSE)) {
     if (cmd.hasOption(Shell.VERBOSE)) {
-      System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-          bucket.getBucketInfo().toJsonString()));
+      OzoneBucket bucket = vol.getBucket(bucketName);
+      System.out.printf(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+          JsonUtils.toJsonString(OzoneClientUtils.asBucketInfo(bucket))));
     }
     }
   }
   }
 }
 }

+ 5 - 15
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java

@@ -19,8 +19,8 @@
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -37,7 +37,6 @@ import java.nio.file.Paths;
 public class DeleteBucketHandler extends Handler {
 public class DeleteBucketHandler extends Handler {
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
-  private String rootName;
 
 
   /**
   /**
    * Executes the Client Calls.
    * Executes the Client Calls.
@@ -52,7 +51,7 @@ public class DeleteBucketHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.DELETE_BUCKET)) {
     if (!cmd.hasOption(Shell.DELETE_BUCKET)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : deleteBucket is missing");
           "Incorrect call : deleteBucket is missing");
     }
     }
 
 
@@ -60,7 +59,7 @@ public class DeleteBucketHandler extends Handler {
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 2) {
     if (path.getNameCount() < 2) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume and bucket name required in delete Bucket");
           "volume and bucket name required in delete Bucket");
     }
     }
 
 
@@ -72,16 +71,7 @@ public class DeleteBucketHandler extends Handler {
       System.out.printf("Bucket Name : %s%n", bucketName);
       System.out.printf("Bucket Name : %s%n", bucketName);
     }
     }
 
 
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    } else {
-      rootName = System.getProperty("user.name");
-    }
-
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
-
-    OzoneVolume vol = client.getVolume(volumeName);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     vol.deleteBucket(bucketName);
     vol.deleteBucket(bucketName);
   }
   }
 }
 }

+ 8 - 17
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java

@@ -19,9 +19,10 @@ package org.apache.hadoop.ozone.web.ozShell.bucket;
 
 
 
 
 import org.apache.commons.cli.CommandLine;
 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.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -39,7 +40,6 @@ import java.nio.file.Paths;
 public class InfoBucketHandler extends Handler {
 public class InfoBucketHandler extends Handler {
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
-  private String rootName;
 
 
   /**
   /**
    * Executes the Client Calls.
    * Executes the Client Calls.
@@ -54,7 +54,7 @@ public class InfoBucketHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.INFO_BUCKET)) {
     if (!cmd.hasOption(Shell.INFO_BUCKET)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : infoBucket is missing");
           "Incorrect call : infoBucket is missing");
     }
     }
 
 
@@ -63,7 +63,7 @@ public class InfoBucketHandler extends Handler {
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
 
 
     if (path.getNameCount() < 2) {
     if (path.getNameCount() < 2) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume and bucket name required in info Bucket");
           "volume and bucket name required in info Bucket");
     }
     }
 
 
@@ -75,20 +75,11 @@ public class InfoBucketHandler extends Handler {
       System.out.printf("Bucket Name : %s%n", bucketName);
       System.out.printf("Bucket Name : %s%n", bucketName);
     }
     }
 
 
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    } else {
-      rootName = System.getProperty("user.name");
-    }
-
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
-
-    OzoneVolume vol = client.getVolume(volumeName);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     OzoneBucket bucket = vol.getBucket(bucketName);
     OzoneBucket bucket = vol.getBucket(bucketName);
 
 
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        bucket.getBucketInfo().toJsonString()));
+        JsonUtils.toJsonString(OzoneClientUtils.asBucketInfo(bucket))));
   }
   }
 
 
 }
 }

+ 25 - 26
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java

@@ -19,13 +19,14 @@
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
 
 import org.apache.commons.cli.CommandLine;
 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.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
-import org.apache.hadoop.ozone.web.response.BucketInfo;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 
 
@@ -34,15 +35,15 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.List;
-import java.util.stream.Collectors;
 
 
 /**
 /**
  * Executes List Bucket.
  * Executes List Bucket.
  */
  */
 public class ListBucketHandler extends Handler {
 public class ListBucketHandler extends Handler {
   private String volumeName;
   private String volumeName;
-  private String rootName;
 
 
   /**
   /**
    * Executes the Client Calls.
    * Executes the Client Calls.
@@ -57,7 +58,7 @@ public class ListBucketHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.LIST_BUCKET)) {
     if (!cmd.hasOption(Shell.LIST_BUCKET)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : listBucket is missing");
           "Incorrect call : listBucket is missing");
     }
     }
 
 
@@ -65,30 +66,20 @@ public class ListBucketHandler extends Handler {
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 1) {
     if (path.getNameCount() < 1) {
-      throw new OzoneRestClientException("volume is required in listBucket");
+      throw new OzoneClientException("volume is required in listBucket");
     }
     }
 
 
     volumeName = path.getName(0).toString();
     volumeName = path.getName(0).toString();
 
 
-
     if (cmd.hasOption(Shell.VERBOSE)) {
     if (cmd.hasOption(Shell.VERBOSE)) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Volume Name : %s%n", volumeName);
     }
     }
 
 
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    } else {
-      rootName = System.getProperty("user.name");
-    }
-
-
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
-
-    String length = null;
+    int maxBuckets = Integer.MAX_VALUE;
     if (cmd.hasOption(Shell.LIST_LENGTH)) {
     if (cmd.hasOption(Shell.LIST_LENGTH)) {
-      length = cmd.getOptionValue(Shell.LIST_LENGTH);
+      String length = cmd.getOptionValue(Shell.LIST_LENGTH);
       OzoneUtils.verifyMaxKeyLength(length);
       OzoneUtils.verifyMaxKeyLength(length);
+      maxBuckets = Integer.parseInt(length);
     }
     }
 
 
     String startBucket = null;
     String startBucket = null;
@@ -101,13 +92,21 @@ public class ListBucketHandler extends Handler {
       prefix = cmd.getOptionValue(Shell.PREFIX);
       prefix = cmd.getOptionValue(Shell.PREFIX);
     }
     }
 
 
-    OzoneVolume vol = client.getVolume(volumeName);
-    List<OzoneBucket> bucketList = vol.listBuckets(length, startBucket, prefix);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
+    Iterator<OzoneBucket> bucketIterator = vol.listBuckets(prefix, startBucket);
+    List<BucketInfo> bucketList = new ArrayList<>();
+    while (maxBuckets > 0 && bucketIterator.hasNext()) {
+      BucketInfo bucketInfo = OzoneClientUtils.asBucketInfo(bucketIterator.next());
+      bucketList.add(bucketInfo);
+      maxBuckets -= 1;
+    }
 
 
-    List<BucketInfo> jsonData = bucketList.stream()
-        .map(OzoneBucket::getBucketInfo).collect(Collectors.toList());
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      System.out.printf("Found : %d buckets for volume : %s ",
+          bucketList.size(), volumeName);
+    }
     System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
     System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        JsonUtils.toJsonString(jsonData)));
+        JsonUtils.toJsonString(bucketList)));
   }
   }
 }
 }
 
 

+ 21 - 20
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java

@@ -18,9 +18,11 @@
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
 
 import org.apache.commons.cli.CommandLine;
 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.OzoneVolume;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -31,6 +33,9 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
 
 
 /**
 /**
  * Allows users to add and remove acls and from a bucket.
  * Allows users to add and remove acls and from a bucket.
@@ -38,13 +43,12 @@ import java.nio.file.Paths;
 public class UpdateBucketHandler extends Handler {
 public class UpdateBucketHandler extends Handler {
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
-  private String rootName;
 
 
   @Override
   @Override
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.UPDATE_BUCKET)) {
     if (!cmd.hasOption(Shell.UPDATE_BUCKET)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : updateBucket is missing");
           "Incorrect call : updateBucket is missing");
     }
     }
 
 
@@ -53,7 +57,7 @@ public class UpdateBucketHandler extends Handler {
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
 
 
     if (path.getNameCount() < 2) {
     if (path.getNameCount() < 2) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume and bucket name required in update bucket");
           "volume and bucket name required in update bucket");
     }
     }
 
 
@@ -65,30 +69,27 @@ public class UpdateBucketHandler extends Handler {
       System.out.printf("Bucket Name : %s%n", bucketName);
       System.out.printf("Bucket Name : %s%n", bucketName);
     }
     }
 
 
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    } else {
-      rootName = System.getProperty("user.name");
-    }
-
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
-
-    OzoneVolume vol = client.getVolume(volumeName);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
     if (cmd.hasOption(Shell.ADD_ACLS)) {
     if (cmd.hasOption(Shell.ADD_ACLS)) {
       String aclString = cmd.getOptionValue(Shell.ADD_ACLS);
       String aclString = cmd.getOptionValue(Shell.ADD_ACLS);
       String[] aclArray = aclString.split(",");
       String[] aclArray = aclString.split(",");
-      vol.addAcls(bucketName, aclArray);
+      List<OzoneAcl> aclList =
+          Arrays.stream(aclArray).map(acl -> OzoneAcl.parseAcl(acl))
+              .collect(Collectors.toList());
+      bucket.addAcls(aclList);
     }
     }
 
 
     if (cmd.hasOption(Shell.REMOVE_ACLS)) {
     if (cmd.hasOption(Shell.REMOVE_ACLS)) {
       String aclString = cmd.getOptionValue(Shell.REMOVE_ACLS);
       String aclString = cmd.getOptionValue(Shell.REMOVE_ACLS);
       String[] aclArray = aclString.split(",");
       String[] aclArray = aclString.split(",");
-      vol.removeAcls(bucketName, aclArray);
+      List<OzoneAcl> aclList =
+          Arrays.stream(aclArray).map(acl -> OzoneAcl.parseAcl(acl))
+              .collect(Collectors.toList());
+      bucket.removeAcls(aclList);
     }
     }
 
 
-    OzoneBucket bucket = vol.getBucket(bucketName);
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        bucket.getBucketInfo().toJsonString()));
+        JsonUtils.toJsonString(OzoneClientUtils.asBucketInfo(bucket))));
   }
   }
 }
 }

+ 6 - 20
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/DeleteKeyHandler.java

@@ -19,9 +19,9 @@
 package org.apache.hadoop.ozone.web.ozShell.keys;
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
 
 import org.apache.commons.cli.CommandLine;
 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.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -36,7 +36,6 @@ import java.nio.file.Paths;
  * Executes Delete Key.
  * Executes Delete Key.
  */
  */
 public class DeleteKeyHandler extends Handler {
 public class DeleteKeyHandler extends Handler {
-  private String userName;
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
   private String keyName;
   private String keyName;
@@ -53,23 +52,15 @@ public class DeleteKeyHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.DELETE_KEY)) {
     if (!cmd.hasOption(Shell.DELETE_KEY)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : deleteKey is missing");
           "Incorrect call : deleteKey is missing");
     }
     }
 
 
-
-    if (cmd.hasOption(Shell.USER)) {
-      userName = cmd.getOptionValue(Shell.USER);
-    } else {
-      userName = System.getProperty("user.name");
-    }
-
-
     String ozoneURIString = cmd.getOptionValue(Shell.DELETE_KEY);
     String ozoneURIString = cmd.getOptionValue(Shell.DELETE_KEY);
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 3) {
     if (path.getNameCount() < 3) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume/bucket/key name required in deleteKey");
           "volume/bucket/key name required in deleteKey");
     }
     }
 
 
@@ -84,13 +75,8 @@ public class DeleteKeyHandler extends Handler {
       System.out.printf("Key Name : %s%n", keyName);
       System.out.printf("Key Name : %s%n", keyName);
     }
     }
 
 
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(userName);
-
-
-    OzoneVolume vol = client.getVolume(volumeName);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     OzoneBucket bucket = vol.getBucket(bucketName);
     OzoneBucket bucket = vol.getBucket(bucketName);
     bucket.deleteKey(keyName);
     bucket.deleteKey(keyName);
-
   }
   }
 }
 }

+ 29 - 17
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java

@@ -20,24 +20,34 @@ package org.apache.hadoop.ozone.web.ozShell.keys;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.Paths;
 
 
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys
+    .OZONE_SCM_CHUNK_SIZE_DEFAULT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys
+    .OZONE_SCM_CHUNK_SIZE_KEY;
+
 /**
 /**
  * Gets an existing key.
  * Gets an existing key.
  */
  */
 public class GetKeyHandler extends Handler {
 public class GetKeyHandler extends Handler {
-  private String userName;
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
   private String keyName;
   private String keyName;
@@ -56,26 +66,19 @@ public class GetKeyHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.GET_KEY)) {
     if (!cmd.hasOption(Shell.GET_KEY)) {
-      throw new OzoneRestClientException("Incorrect call : getKey is missing");
+      throw new OzoneClientException("Incorrect call : getKey is missing");
     }
     }
 
 
     if (!cmd.hasOption(Shell.FILE)) {
     if (!cmd.hasOption(Shell.FILE)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "get key needs a file path to download to");
           "get key needs a file path to download to");
     }
     }
 
 
-    if (cmd.hasOption(Shell.USER)) {
-      userName = cmd.getOptionValue(Shell.USER);
-    } else {
-      userName = System.getProperty("user.name");
-    }
-
-
     String ozoneURIString = cmd.getOptionValue(Shell.GET_KEY);
     String ozoneURIString = cmd.getOptionValue(Shell.GET_KEY);
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 3) {
     if (path.getNameCount() < 3) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume/bucket/key name required in putKey");
           "volume/bucket/key name required in putKey");
     }
     }
 
 
@@ -97,19 +100,28 @@ public class GetKeyHandler extends Handler {
 
 
 
 
     if (dataFile.exists()) {
     if (dataFile.exists()) {
-      throw new OzoneRestClientException(fileName +
+      throw new OzoneClientException(fileName +
                                          "exists. Download will overwrite an " +
                                          "exists. Download will overwrite an " +
                                          "existing file. Aborting.");
                                          "existing file. Aborting.");
     }
     }
 
 
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(userName);
-
-    client.getKey(volumeName, bucketName, keyName, dataFilePath);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    OzoneInputStream keyInputStream = bucket.readKey(keyName);
+    if (dataFilePath != null) {
+      FileOutputStream outputStream = new FileOutputStream(dataFile);
+      IOUtils.copyBytes(keyInputStream, outputStream, new OzoneConfiguration()
+          .getInt(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT));
+      outputStream.close();
+    } else {
+      throw new OzoneClientException(
+          "Can not access the file \"" + fileName + "\"");
+    }
     if(cmd.hasOption(Shell.VERBOSE)) {
     if(cmd.hasOption(Shell.VERBOSE)) {
       FileInputStream stream = new FileInputStream(dataFile);
       FileInputStream stream = new FileInputStream(dataFile);
       String hash = DigestUtils.md5Hex(stream);
       String hash = DigestUtils.md5Hex(stream);
       System.out.printf("Downloaded file hash : %s%n", hash);
       System.out.printf("Downloaded file hash : %s%n", hash);
+      stream.close();
     }
     }
 
 
   }
   }

+ 10 - 21
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java

@@ -25,10 +25,11 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.Paths;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneBucket;
-import org.apache.hadoop.ozone.web.client.OzoneKey;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -38,7 +39,6 @@ import org.apache.hadoop.ozone.web.utils.JsonUtils;
  * Executes Info Object.
  * Executes Info Object.
  */
  */
 public class InfoKeyHandler extends Handler {
 public class InfoKeyHandler extends Handler {
-  private String userName;
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
   private String keyName;
   private String keyName;
@@ -55,22 +55,14 @@ public class InfoKeyHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.INFO_KEY)) {
     if (!cmd.hasOption(Shell.INFO_KEY)) {
-      throw new OzoneRestClientException("Incorrect call : infoKey is missing");
+      throw new OzoneClientException("Incorrect call : infoKey is missing");
     }
     }
 
 
-
-    if (cmd.hasOption(Shell.USER)) {
-      userName = cmd.getOptionValue(Shell.USER);
-    } else {
-      userName = System.getProperty("user.name");
-    }
-
-
     String ozoneURIString = cmd.getOptionValue(Shell.INFO_KEY);
     String ozoneURIString = cmd.getOptionValue(Shell.INFO_KEY);
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 3) {
     if (path.getNameCount() < 3) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume/bucket/key name required in infoKey");
           "volume/bucket/key name required in infoKey");
     }
     }
 
 
@@ -85,14 +77,11 @@ public class InfoKeyHandler extends Handler {
       System.out.printf("Key Name : %s%n", keyName);
       System.out.printf("Key Name : %s%n", keyName);
     }
     }
 
 
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(userName);
-
-    OzoneVolume vol = client.getVolume(volumeName);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     OzoneBucket bucket = vol.getBucket(bucketName);
     OzoneBucket bucket = vol.getBucket(bucketName);
-    OzoneKey key = bucket.getKeyInfo(keyName);
+    OzoneKey key = bucket.getKey(keyName);
 
 
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        key.getObjectInfo().toJsonString()));
+        JsonUtils.toJsonString(OzoneClientUtils.asKeyInfo(key))));
   }
   }
 }
 }

+ 24 - 24
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java

@@ -19,12 +19,11 @@
 package org.apache.hadoop.ozone.web.ozShell.keys;
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
-import org.apache.hadoop.ozone.web.client.OzoneKey;
+import org.apache.hadoop.ozone.client.*;
+import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
-import org.apache.hadoop.ozone.web.response.KeyInfo;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 
 
@@ -33,14 +32,14 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.List;
-import java.util.stream.Collectors;
 
 
 /**
 /**
  * Executes List Keys.
  * Executes List Keys.
  */
  */
 public class ListKeyHandler extends Handler {
 public class ListKeyHandler extends Handler {
-  private String userName;
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
 
 
@@ -57,14 +56,15 @@ public class ListKeyHandler extends Handler {
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
 
 
     if (!cmd.hasOption(Shell.LIST_KEY)) {
     if (!cmd.hasOption(Shell.LIST_KEY)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : listKey is missing");
           "Incorrect call : listKey is missing");
     }
     }
 
 
-    String length = null;
+    int maxKeys = Integer.MAX_VALUE;
     if (cmd.hasOption(Shell.LIST_LENGTH)) {
     if (cmd.hasOption(Shell.LIST_LENGTH)) {
-      length = cmd.getOptionValue(Shell.LIST_LENGTH);
+      String length = cmd.getOptionValue(Shell.LIST_LENGTH);
       OzoneUtils.verifyMaxKeyLength(length);
       OzoneUtils.verifyMaxKeyLength(length);
+      maxKeys = Integer.parseInt(length);
     }
     }
 
 
     String startKey = null;
     String startKey = null;
@@ -81,7 +81,7 @@ public class ListKeyHandler extends Handler {
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 2) {
     if (path.getNameCount() < 2) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume/bucket is required in listKey");
           "volume/bucket is required in listKey");
     }
     }
 
 
@@ -94,23 +94,23 @@ public class ListKeyHandler extends Handler {
       System.out.printf("bucket Name : %s%n", bucketName);
       System.out.printf("bucket Name : %s%n", bucketName);
     }
     }
 
 
-    if (cmd.hasOption(Shell.USER)) {
-      userName = cmd.getOptionValue(Shell.USER);
-    } else {
-      userName = System.getProperty("user.name");
-    }
-
-
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(userName);
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    Iterator<OzoneKey> keyIterator = bucket.listKeys(prefix, startKey);
+    List<KeyInfo> keyInfos = new ArrayList<>();
 
 
-    List<OzoneKey> keys = client.listKeys(volumeName, bucketName, length,
-        startKey, prefix);
+    while (maxKeys > 0 && keyIterator.hasNext()) {
+      KeyInfo key = OzoneClientUtils.asKeyInfo(keyIterator.next());
+      keyInfos.add(key);
+      maxKeys -= 1;
+    }
 
 
-    List<KeyInfo> jsonData = keys.stream()
-        .map(OzoneKey::getObjectInfo).collect(Collectors.toList());
-    System.out.printf(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        JsonUtils.toJsonString(jsonData)));
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      System.out.printf("Found : %d keys for bucket %s in volume : %s ",
+          keyInfos.size(), bucketName, volumeName);
+    }
+    System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+        JsonUtils.toJsonString(keyInfos)));
   }
   }
 
 
 }
 }

+ 33 - 16
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java

@@ -20,7 +20,15 @@ package org.apache.hadoop.ozone.web.ozShell.keys;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -33,11 +41,15 @@ import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.Paths;
 
 
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
+
 /**
 /**
  * Puts a file into an ozone bucket.
  * Puts a file into an ozone bucket.
  */
  */
 public class PutKeyHandler extends Handler {
 public class PutKeyHandler extends Handler {
-  private String userName;
   private String volumeName;
   private String volumeName;
   private String bucketName;
   private String bucketName;
   private String keyName;
   private String keyName;
@@ -54,24 +66,18 @@ public class PutKeyHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.PUT_KEY)) {
     if (!cmd.hasOption(Shell.PUT_KEY)) {
-      throw new OzoneRestClientException("Incorrect call : putKey is missing");
+      throw new OzoneClientException("Incorrect call : putKey is missing");
     }
     }
 
 
     if (!cmd.hasOption(Shell.FILE)) {
     if (!cmd.hasOption(Shell.FILE)) {
-      throw new OzoneRestClientException("put key needs a file to put");
-    }
-
-    if (cmd.hasOption(Shell.USER)) {
-      userName = cmd.getOptionValue(Shell.USER);
-    } else {
-      userName = System.getProperty("user.name");
+      throw new OzoneClientException("put key needs a file to put");
     }
     }
 
 
     String ozoneURIString = cmd.getOptionValue(Shell.PUT_KEY);
     String ozoneURIString = cmd.getOptionValue(Shell.PUT_KEY);
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     Path path = Paths.get(ozoneURI.getPath());
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 3) {
     if (path.getNameCount() < 3) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "volume/bucket/key name required in putKey");
           "volume/bucket/key name required in putKey");
     }
     }
 
 
@@ -86,7 +92,6 @@ public class PutKeyHandler extends Handler {
       System.out.printf("Key Name : %s%n", keyName);
       System.out.printf("Key Name : %s%n", keyName);
     }
     }
 
 
-
     String fileName = cmd.getOptionValue(Shell.FILE);
     String fileName = cmd.getOptionValue(Shell.FILE);
     File dataFile = new File(fileName);
     File dataFile = new File(fileName);
 
 
@@ -97,10 +102,22 @@ public class PutKeyHandler extends Handler {
       stream.close();
       stream.close();
     }
     }
 
 
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(userName);
-
-    client.putKey(volumeName, bucketName, keyName, dataFile);
+    Configuration conf = new OzoneConfiguration();
+    ReplicationFactor replicationFactor = ReplicationFactor.valueOf(
+        conf.getInt(OZONE_REPLICATION, ReplicationFactor.THREE.getValue()));
+    ReplicationType replicationType = ReplicationType.valueOf(
+        conf.get(OZONE_REPLICATION_TYPE, ReplicationType.RATIS.toString()));
+
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    OzoneOutputStream outputStream = bucket
+        .createKey(keyName, dataFile.length(), replicationType,
+            replicationFactor);
+    FileInputStream fileInputStream = new FileInputStream(dataFile);
+    IOUtils.copyBytes(fileInputStream, outputStream,
+        conf.getInt(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT));
+    outputStream.close();
+    fileInputStream.close();
   }
   }
 
 
 }
 }

+ 18 - 10
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.ozone.web.ozShell.volume;
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -52,14 +54,14 @@ public class CreateVolumeHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.CREATE_VOLUME)) {
     if (!cmd.hasOption(Shell.CREATE_VOLUME)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : createVolume is missing");
           "Incorrect call : createVolume is missing");
     }
     }
 
 
     String ozoneURIString = cmd.getOptionValue(Shell.CREATE_VOLUME);
     String ozoneURIString = cmd.getOptionValue(Shell.CREATE_VOLUME);
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     if (ozoneURI.getPath().isEmpty()) {
     if (ozoneURI.getPath().isEmpty()) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Volume name is required to create a volume");
           "Volume name is required to create a volume");
     }
     }
 
 
@@ -77,7 +79,7 @@ public class CreateVolumeHandler extends Handler {
     }
     }
 
 
     if (!cmd.hasOption(Shell.USER)) {
     if (!cmd.hasOption(Shell.USER)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "User name is needed in createVolume call.");
           "User name is needed in createVolume call.");
     }
     }
 
 
@@ -86,13 +88,19 @@ public class CreateVolumeHandler extends Handler {
     }
     }
 
 
     userName = cmd.getOptionValue(Shell.USER);
     userName = cmd.getOptionValue(Shell.USER);
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
 
 
-    OzoneVolume vol = client.createVolume(volumeName, userName, quota);
+    VolumeArgs.Builder volumeArgsBuilder = VolumeArgs.newBuilder()
+        .setAdmin(rootName)
+        .setOwner(userName);
+    if (quota != null) {
+      volumeArgsBuilder.setQuota(quota);
+    }
+    client.getObjectStore().createVolume(volumeName, volumeArgsBuilder.build());
+
     if (cmd.hasOption(Shell.VERBOSE)) {
     if (cmd.hasOption(Shell.VERBOSE)) {
-      System.out.printf("%s%n",
-          JsonUtils.toJsonStringWithDefaultPrettyPrinter(vol.getJsonString()));
+      OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
+      System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+          JsonUtils.toJsonString(OzoneClientUtils.asVolumeInfo(vol))));
     }
     }
   }
   }
 }
 }

+ 4 - 14
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.ozone.web.ozShell.volume;
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -34,7 +34,6 @@ import java.net.URISyntaxException;
 public class DeleteVolumeHandler extends Handler {
 public class DeleteVolumeHandler extends Handler {
 
 
   private String volumeName;
   private String volumeName;
-  private String rootName;
 
 
   /**
   /**
    * Executes the delete volume call.
    * Executes the delete volume call.
@@ -49,14 +48,14 @@ public class DeleteVolumeHandler extends Handler {
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
 
 
     if (!cmd.hasOption(Shell.DELETE_VOLUME)) {
     if (!cmd.hasOption(Shell.DELETE_VOLUME)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : deleteVolume call is missing");
           "Incorrect call : deleteVolume call is missing");
     }
     }
 
 
     String ozoneURIString = cmd.getOptionValue(Shell.DELETE_VOLUME);
     String ozoneURIString = cmd.getOptionValue(Shell.DELETE_VOLUME);
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     if (ozoneURI.getPath().isEmpty()) {
     if (ozoneURI.getPath().isEmpty()) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Volume name is required to delete a volume");
           "Volume name is required to delete a volume");
     }
     }
 
 
@@ -67,15 +66,6 @@ public class DeleteVolumeHandler extends Handler {
       System.out.printf("Volume name : %s%n", volumeName);
       System.out.printf("Volume name : %s%n", volumeName);
     }
     }
 
 
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    } else {
-      rootName = System.getProperty("user.name");
-    }
-
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
-    client.deleteVolume(volumeName);
-
+    client.getObjectStore().deleteVolume(volumeName);
   }
   }
 }
 }

+ 8 - 30
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java

@@ -19,8 +19,9 @@
 package org.apache.hadoop.ozone.web.ozShell.volume;
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -35,9 +36,7 @@ import java.net.URISyntaxException;
  */
  */
 public class InfoVolumeHandler extends Handler{
 public class InfoVolumeHandler extends Handler{
 
 
-  private String rootName;
   private String volumeName;
   private String volumeName;
-  private String userName;
 
 
   /**
   /**
    * Executes volume Info.
    * Executes volume Info.
@@ -53,43 +52,22 @@ public class InfoVolumeHandler extends Handler{
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
 
 
     if (!cmd.hasOption(Shell.INFO_VOLUME)) {
     if (!cmd.hasOption(Shell.INFO_VOLUME)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : infoVolume is missing");
           "Incorrect call : infoVolume is missing");
     }
     }
 
 
     String ozoneURIString = cmd.getOptionValue(Shell.INFO_VOLUME);
     String ozoneURIString = cmd.getOptionValue(Shell.INFO_VOLUME);
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     if (ozoneURI.getPath().isEmpty()) {
     if (ozoneURI.getPath().isEmpty()) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Volume name is required to get info of a volume");
           "Volume name is required to get info of a volume");
     }
     }
 
 
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    }
-
     // we need to skip the slash in the URI path
     // we need to skip the slash in the URI path
     volumeName = ozoneURI.getPath().substring(1);
     volumeName = ozoneURI.getPath().substring(1);
 
 
-    if (cmd.hasOption(Shell.USER)) {
-      userName = cmd.getOptionValue(Shell.USER);
-    } else {
-      userName = System.getProperty("user.name");
-    }
-
-    client.setEndPointURI(ozoneURI);
-
-    if (rootName != null) {
-      client.setUserAuth(rootName);
-    } else {
-      client.setUserAuth(userName);
-    }
-
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
-
-    OzoneVolume vol = client.getVolume(volumeName);
-    System.out.printf("%s%n",
-        JsonUtils.toJsonStringWithDefaultPrettyPrinter(vol.getJsonString()));
+    OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
+    System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+        JsonUtils.toJsonString(OzoneClientUtils.asVolumeInfo(vol))));
   }
   }
 }
 }

+ 28 - 30
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java

@@ -19,26 +19,26 @@
 package org.apache.hadoop.ozone.web.ozShell.volume;
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.List;
-import java.util.stream.Collectors;
 
 
 /**
 /**
  * Executes List Volume call.
  * Executes List Volume call.
  */
  */
 public class ListVolumeHandler extends Handler {
 public class ListVolumeHandler extends Handler {
-  private String rootName;
   private String userName;
   private String userName;
 
 
   /**
   /**
@@ -54,16 +54,16 @@ public class ListVolumeHandler extends Handler {
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
 
 
     if (!cmd.hasOption(Shell.LIST_VOLUME)) {
     if (!cmd.hasOption(Shell.LIST_VOLUME)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : listVolume is missing");
           "Incorrect call : listVolume is missing");
     }
     }
 
 
-    int maxKeys = 0;
+    int maxVolumes = Integer.MAX_VALUE;
     if (cmd.hasOption(Shell.LIST_LENGTH)) {
     if (cmd.hasOption(Shell.LIST_LENGTH)) {
       String length = cmd.getOptionValue(Shell.LIST_LENGTH);
       String length = cmd.getOptionValue(Shell.LIST_LENGTH);
       OzoneUtils.verifyMaxKeyLength(length);
       OzoneUtils.verifyMaxKeyLength(length);
 
 
-      maxKeys = Integer.parseInt(length);
+      maxVolumes = Integer.parseInt(length);
     }
     }
 
 
     String startVolume = null;
     String startVolume = null;
@@ -77,11 +77,7 @@ public class ListVolumeHandler extends Handler {
     }
     }
 
 
     String ozoneURIString = cmd.getOptionValue(Shell.LIST_VOLUME);
     String ozoneURIString = cmd.getOptionValue(Shell.LIST_VOLUME);
-    URI ozoneURI = verifyURI(ozoneURIString);
-
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    }
+    verifyURI(ozoneURIString);
 
 
     if (cmd.hasOption(Shell.USER)) {
     if (cmd.hasOption(Shell.USER)) {
       userName = cmd.getOptionValue(Shell.USER);
       userName = cmd.getOptionValue(Shell.USER);
@@ -89,26 +85,28 @@ public class ListVolumeHandler extends Handler {
       userName = System.getProperty("user.name");
       userName = System.getProperty("user.name");
     }
     }
 
 
-    client.setEndPointURI(ozoneURI);
-    if (rootName != null) {
-      client.setUserAuth(rootName);
+    Iterator<OzoneVolume> volumeIterator;
+    if(userName != null) {
+      volumeIterator = client.getObjectStore()
+          .listVolumesByUser(userName, prefix, startVolume);
     } else {
     } else {
-      client.setUserAuth(userName);
+      volumeIterator = client.getObjectStore().listVolumes(prefix);
+    }
+
+    List<VolumeInfo> volumeInfos = new ArrayList<>();
+
+    while (maxVolumes > 0 && volumeIterator.hasNext()) {
+      VolumeInfo volume = OzoneClientUtils.asVolumeInfo(volumeIterator.next());
+      volumeInfos.add(volume);
+      maxVolumes -= 1;
     }
     }
 
 
-    List<OzoneVolume> volumes = client.listVolumes(userName, prefix, maxKeys,
-        startVolume);
-    if (volumes != null) {
-      if (cmd.hasOption(Shell.VERBOSE)) {
-        System.out.printf("Found : %d volumes for user : %s %n", volumes.size(),
-            userName);
-      }
-
-      List<VolumeInfo> jsonData = volumes.stream()
-          .map(OzoneVolume::getVolumeInfo).collect(Collectors.toList());
-      System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-          JsonUtils.toJsonString(jsonData)));
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      System.out.printf("Found : %d volumes for user : %s ", volumeInfos.size(),
+          userName);
     }
     }
+    System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+        JsonUtils.toJsonString(volumeInfos)));
   }
   }
 }
 }
 
 

+ 8 - 16
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java

@@ -19,7 +19,9 @@
 package org.apache.hadoop.ozone.web.ozShell.volume;
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.web.client.OzoneRestClientException;
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -32,7 +34,6 @@ import java.net.URISyntaxException;
  * Executes update volume calls.
  * Executes update volume calls.
  */
  */
 public class UpdateVolumeHandler extends Handler {
 public class UpdateVolumeHandler extends Handler {
-  private String rootName;
   private String ownerName;
   private String ownerName;
   private String volumeName;
   private String volumeName;
   private String quota;
   private String quota;
@@ -49,26 +50,20 @@ public class UpdateVolumeHandler extends Handler {
   protected void execute(CommandLine cmd)
   protected void execute(CommandLine cmd)
       throws IOException, OzoneException, URISyntaxException {
       throws IOException, OzoneException, URISyntaxException {
     if (!cmd.hasOption(Shell.UPDATE_VOLUME)) {
     if (!cmd.hasOption(Shell.UPDATE_VOLUME)) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Incorrect call : updateVolume is missing");
           "Incorrect call : updateVolume is missing");
     }
     }
 
 
     String ozoneURIString = cmd.getOptionValue(Shell.UPDATE_VOLUME);
     String ozoneURIString = cmd.getOptionValue(Shell.UPDATE_VOLUME);
     URI ozoneURI = verifyURI(ozoneURIString);
     URI ozoneURI = verifyURI(ozoneURIString);
     if (ozoneURI.getPath().isEmpty()) {
     if (ozoneURI.getPath().isEmpty()) {
-      throw new OzoneRestClientException(
+      throw new OzoneClientException(
           "Volume name is required to update a volume");
           "Volume name is required to update a volume");
     }
     }
 
 
     // we need to skip the slash in the URI path
     // we need to skip the slash in the URI path
     volumeName = ozoneURI.getPath().substring(1);
     volumeName = ozoneURI.getPath().substring(1);
 
 
-    if (cmd.hasOption(Shell.RUNAS)) {
-      rootName = "hdfs";
-    } else {
-      rootName = System.getProperty("user.name");
-    }
-
     if (cmd.hasOption(Shell.QUOTA)) {
     if (cmd.hasOption(Shell.QUOTA)) {
       quota = cmd.getOptionValue(Shell.QUOTA);
       quota = cmd.getOptionValue(Shell.QUOTA);
     }
     }
@@ -77,16 +72,13 @@ public class UpdateVolumeHandler extends Handler {
       ownerName = cmd.getOptionValue(Shell.USER);
       ownerName = cmd.getOptionValue(Shell.USER);
     }
     }
 
 
-    client.setEndPointURI(ozoneURI);
-    client.setUserAuth(rootName);
-
+    OzoneVolume volume = client.getObjectStore().getVolume(volumeName);
     if (quota != null && !quota.isEmpty()) {
     if (quota != null && !quota.isEmpty()) {
-      client.setVolumeQuota(volumeName, quota);
+      volume.setQuota(OzoneQuota.parseQuota(quota));
     }
     }
 
 
     if (ownerName != null && !ownerName.isEmpty()) {
     if (ownerName != null && !ownerName.isEmpty()) {
-      client.setVolumeOwner(volumeName, ownerName);
+      volume.setOwner(ownerName);
     }
     }
-
   }
   }
 }
 }

+ 0 - 6
hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/Constants.java

@@ -23,12 +23,8 @@ package org.apache.hadoop.fs.ozone;
  */
  */
 public final class Constants {
 public final class Constants {
 
 
-  public static final String OZONE_URI_SCHEME = "o3";
-
   public static final String OZONE_DEFAULT_USER = "hdfs";
   public static final String OZONE_DEFAULT_USER = "hdfs";
 
 
-  public static final String OZONE_HTTP_SCHEME = "http://";
-
   public static final String OZONE_USER_DIR = "/user";
   public static final String OZONE_USER_DIR = "/user";
 
 
   /** Local buffer directory. */
   /** Local buffer directory. */
@@ -37,8 +33,6 @@ public final class Constants {
   /** Temporary directory. */
   /** Temporary directory. */
   public static final String BUFFER_TMP_KEY = "hadoop.tmp.dir";
   public static final String BUFFER_TMP_KEY = "hadoop.tmp.dir";
 
 
-  public static final String OZONE_URI_DELIMITER = "/";
-
   /** Page size for Ozone listing operation. */
   /** Page size for Ozone listing operation. */
   public static final int LISTING_PAGE_SIZE = 1024;
   public static final int LISTING_PAGE_SIZE = 1024;
 
 

+ 2 - 1
hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/OzFs.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DelegateToFileSystem;
 import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.ozone.OzoneConsts;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
@@ -38,6 +39,6 @@ public class OzFs extends DelegateToFileSystem {
   public OzFs(URI theUri, Configuration conf)
   public OzFs(URI theUri, Configuration conf)
       throws IOException, URISyntaxException {
       throws IOException, URISyntaxException {
     super(theUri, new OzoneFileSystem(), conf,
     super(theUri, new OzoneFileSystem(), conf,
-        Constants.OZONE_URI_SCHEME, false);
+        OzoneConsts.OZONE_URI_SCHEME, false);
   }
   }
 }
 }

+ 2 - 2
hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java

@@ -64,9 +64,9 @@ import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 
 
 import static org.apache.hadoop.fs.ozone.Constants.OZONE_DEFAULT_USER;
 import static org.apache.hadoop.fs.ozone.Constants.OZONE_DEFAULT_USER;
-import static org.apache.hadoop.fs.ozone.Constants.OZONE_URI_SCHEME;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
 import static org.apache.hadoop.fs.ozone.Constants.OZONE_USER_DIR;
 import static org.apache.hadoop.fs.ozone.Constants.OZONE_USER_DIR;
-import static org.apache.hadoop.fs.ozone.Constants.OZONE_URI_DELIMITER;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
 import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
 import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
 
 
 /**
 /**

+ 2 - 1
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
@@ -94,7 +95,7 @@ public class TestOzoneFSInputStream {
 
 
     // Set the fs.defaultFS and start the filesystem
     // Set the fs.defaultFS and start the filesystem
     String uri = String.format("%s://%s.%s/",
     String uri = String.format("%s://%s.%s/",
-        Constants.OZONE_URI_SCHEME, bucketName, volumeName);
+        OzoneConsts.OZONE_URI_SCHEME, bucketName, volumeName);
     conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, uri);
     conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, uri);
     fs =  FileSystem.get(conf);
     fs =  FileSystem.get(conf);
     int fileLen = 100 * 1024 * 1024;
     int fileLen = 100 * 1024 * 1024;

+ 3 - 2
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java

@@ -24,6 +24,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 
 
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runner.RunWith;
@@ -118,7 +119,7 @@ public class TestOzoneFileInterfaces {
     storageHandler.createBucket(bucketArgs);
     storageHandler.createBucket(bucketArgs);
 
 
     rootPath = String
     rootPath = String
-        .format("%s://%s.%s/", Constants.OZONE_URI_SCHEME, bucketName,
+        .format("%s://%s.%s/", OzoneConsts.OZONE_URI_SCHEME, bucketName,
             volumeName);
             volumeName);
     if (setDefaultFs) {
     if (setDefaultFs) {
       // Set the fs.defaultFS and start the filesystem
       // Set the fs.defaultFS and start the filesystem
@@ -145,7 +146,7 @@ public class TestOzoneFileInterfaces {
           "The initialized file system is not OzoneFileSystem but " +
           "The initialized file system is not OzoneFileSystem but " +
               fs.getClass(),
               fs.getClass(),
           fs instanceof OzoneFileSystem);
           fs instanceof OzoneFileSystem);
-      assertEquals(Constants.OZONE_URI_SCHEME, fs.getUri().getScheme());
+      assertEquals(OzoneConsts.OZONE_URI_SCHEME, fs.getUri().getScheme());
     }
     }
   }
   }
 
 

+ 3 - 3
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java

@@ -24,9 +24,9 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.ozone.Constants;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
@@ -56,7 +56,7 @@ class OzoneContract extends AbstractFSContract {
 
 
   @Override
   @Override
   public String getScheme() {
   public String getScheme() {
-    return Constants.OZONE_URI_SCHEME;
+    return OzoneConsts.OZONE_URI_SCHEME;
   }
   }
 
 
   @Override
   @Override
@@ -107,7 +107,7 @@ class OzoneContract extends AbstractFSContract {
     }
     }
 
 
     String uri = String.format("%s://%s.%s/",
     String uri = String.format("%s://%s.%s/",
-        Constants.OZONE_URI_SCHEME, bucketName, volumeName);
+        OzoneConsts.OZONE_URI_SCHEME, bucketName, volumeName);
     getConf().set("fs.defaultFS", uri);
     getConf().set("fs.defaultFS", uri);
     copyClusterConfigs(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY);
     copyClusterConfigs(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY);
     copyClusterConfigs(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
     copyClusterConfigs(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);