فهرست منبع

HDDS-682. Unified o3 address parsing for ozonen sh. Contributed by Elek, Marton.

Hanisha Koneru 6 سال پیش
والد
کامیت
38a65e3b7c
26فایلهای تغییر یافته به همراه574 افزوده شده و 414 حذف شده
  1. 4 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
  2. 4 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericParentCommand.java
  3. 45 32
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
  4. 4 135
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
  5. 251 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/OzoneAddress.java
  6. 6 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/BucketCommands.java
  7. 7 20
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java
  8. 7 13
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
  9. 7 21
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
  10. 7 19
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
  11. 11 44
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/S3BucketMapping.java
  12. 7 13
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java
  13. 9 14
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/DeleteKeyHandler.java
  14. 8 10
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java
  15. 8 20
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
  16. 6 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/KeyCommands.java
  17. 8 20
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
  18. 8 13
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java
  19. 8 20
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
  20. 7 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
  21. 7 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
  22. 11 16
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
  23. 7 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
  24. 6 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/VolumeCommands.java
  25. 100 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/web/ozShell/TestOzoneAddress.java
  26. 21 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/web/ozShell/package-info.java

+ 4 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java

@@ -62,7 +62,9 @@ public class GenericCli implements Callable<Void>, GenericParentCommand {
   }
 
   private void printError(Throwable error) {
-    if (verbose) {
+    //message could be null in case of NPE. This is unexpected so we can
+    //print out the stack trace.
+    if (verbose || error.getMessage() == null) {
       error.printStackTrace(System.err);
     } else {
       System.err.println(error.getMessage().split("\n")[0]);
@@ -77,6 +79,7 @@ public class GenericCli implements Callable<Void>, GenericParentCommand {
     throw new MissingSubcommandException(cmd.getUsageMessage());
   }
 
+  @Override
   public OzoneConfiguration createOzoneConfiguration() {
     OzoneConfiguration ozoneConf = new OzoneConfiguration();
     if (configurationOverrides != null) {

+ 4 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericParentCommand.java

@@ -16,10 +16,14 @@
  */
 package org.apache.hadoop.hdds.cli;
 
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+
 /**
  * Interface to access the higher level parameters.
  */
 public interface GenericParentCommand {
 
   boolean isVerbose();
+
+  OzoneConfiguration createOzoneConfiguration();
 }

+ 45 - 32
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.ozone.ozShell;
 
-import com.google.common.base.Strings;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
@@ -33,7 +32,6 @@ import java.util.Random;
 import java.util.UUID;
 import java.util.stream.Collectors;
 
-import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.cli.MissingSubcommandException;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
@@ -63,9 +61,18 @@ import org.apache.hadoop.ozone.web.response.VolumeInfo;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.RandomStringUtils;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -82,12 +89,6 @@ import picocli.CommandLine.ParameterException;
 import picocli.CommandLine.ParseResult;
 import picocli.CommandLine.RunLast;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 /**
  * This test class specified for testing Ozone shell command.
  */
@@ -209,8 +210,7 @@ public class TestOzoneShell {
     testCreateVolume(volumeName, "");
     volumeName = "volume" + RandomStringUtils.randomNumeric(5);
     testCreateVolume("/////" + volumeName, "");
-    testCreateVolume("/////", "Volume name is required " +
-        "to create a volume");
+    testCreateVolume("/////", "Volume name is required");
     testCreateVolume("/////vol/123",
         "Invalid volume name. Delimiters (/) not allowed in volume name");
   }
@@ -1126,36 +1126,49 @@ public class TestOzoneShell {
 
   @Test
   public void testS3BucketMapping() throws  IOException {
+
+    List<ServiceInfo> services =
+        cluster.getOzoneManager().getServiceList();
+
+    String omHostName = services.stream().filter(
+        a -> a.getNodeType().equals(HddsProtos.NodeType.OM))
+        .collect(Collectors.toList()).get(0).getHostname();
+
+    String omPort = cluster.getOzoneManager().getRpcPort();
+    String setOmAddress =
+        "--set=" + OZONE_OM_ADDRESS_KEY + "=" + omHostName + ":" + omPort;
+
     String s3Bucket = "bucket1";
     String commandOutput;
     createS3Bucket("ozone", s3Bucket);
+
+    //WHEN
+    String[] args =
+        new String[] {setOmAddress, "bucket",
+            "path", s3Bucket};
+    execute(shell, args);
+
+    //THEN
+    commandOutput = out.toString();
     String volumeName = client.getOzoneVolumeName(s3Bucket);
-    String[] args = new String[] {"bucket", "path", url + "/" + s3Bucket};
-    if (url.startsWith("o3")) {
-      execute(shell, args);
-      commandOutput = out.toString();
-      assertTrue(commandOutput.contains("Volume name for S3Bucket is : " +
-          volumeName));
-      assertTrue(commandOutput.contains(OzoneConsts.OZONE_URI_SCHEME +"://" +
-          s3Bucket + "." + volumeName));
-      out.reset();
-      //Trying to get map for an unknown bucket
-      args = new String[] {"bucket", "path", url + "/" + "unknownbucket"};
-      executeWithError(shell, args, "S3_BUCKET_NOT_FOUND");
-    } else {
-      executeWithError(shell, args, "Ozone REST protocol does not support " +
-          "this operation");
-    }
+    assertTrue(commandOutput.contains("Volume name for S3Bucket is : " +
+        volumeName));
+    assertTrue(commandOutput.contains(OzoneConsts.OZONE_URI_SCHEME + "://" +
+        s3Bucket + "." + volumeName));
+    out.reset();
+
+    //Trying to get map for an unknown bucket
+    args = new String[] {setOmAddress, "bucket", "path",
+        "unknownbucket"};
+    executeWithError(shell, args, "S3_BUCKET_NOT_FOUND");
 
     // No bucket name
-    args = new String[] {"bucket", "path", url};
-    executeWithError(shell, args, "S3Bucket name is required");
+    args = new String[] {setOmAddress, "bucket", "path"};
+    executeWithError(shell, args, "Missing required parameter");
 
     // Invalid bucket name
-    args = new String[] {"bucket", "path", url + "/" + s3Bucket +
-          "/multipleslash"};
-    executeWithError(shell, args, "Invalid S3Bucket name. Delimiters (/) not" +
-        " allowed");
+    args = new String[] {setOmAddress, "bucket", "path", "/asd/multipleslash"};
+    executeWithError(shell, args, "S3_BUCKET_NOT_FOUND");
   }
 
   private void createS3Bucket(String userName, String s3Bucket) {

+ 4 - 135
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java

@@ -18,25 +18,12 @@
 
 package org.apache.hadoop.ozone.web.ozShell;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.concurrent.Callable;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.cli.GenericParentCommand;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.OzoneClientFactory;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_HTTP_SCHEME;
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_RPC_SCHEME;
-import org.apache.http.client.utils.URIBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import picocli.CommandLine.Command;
@@ -51,8 +38,6 @@ public abstract class Handler implements Callable<Void> {
 
   protected static final Logger LOG = LoggerFactory.getLogger(Handler.class);
 
-  protected OzoneClient client;
-
   @ParentCommand
   private GenericParentCommand parent;
 
@@ -61,128 +46,12 @@ public abstract class Handler implements Callable<Void> {
     throw new UnsupportedOperationException();
   }
 
-  /**
-   * verifies user provided URI.
-   *
-   * @param uri - UriString
-   * @return URI
-   * @throws URISyntaxException
-   * @throws OzoneException
-   */
-  protected URI verifyURI(String uri)
-      throws URISyntaxException, OzoneException, IOException {
-    if ((uri == null) || uri.isEmpty()) {
-      throw new OzoneClientException(
-          "Ozone URI is needed to execute this command.");
-    }
-    URIBuilder ozoneURI = new URIBuilder(stringToUri(uri));
-    if (ozoneURI.getPort() == 0) {
-      ozoneURI.setPort(Shell.DEFAULT_OZONE_PORT);
-    }
-
-    Configuration conf = new OzoneConfiguration();
-    String scheme = ozoneURI.getScheme();
-    if (ozoneURI.getScheme() == null || scheme.isEmpty()) {
-      scheme = OZONE_RPC_SCHEME;
-    }
-    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_RPC_SCHEME)) {
-      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();
-  }
-
-  /** Construct a URI from a String with unescaped special characters
-   *  that have non-standard semantics. e.g. /, ?, #. A custom parsing
-   *  is needed to prevent misbehavior.
-   *  @param pathString The input path in string form
-   *  @return URI
-   */
-  private static URI stringToUri(String pathString) throws IOException {
-    // parse uri components
-    String scheme = null;
-    String authority = null;
-    int start = 0;
-
-    // parse uri scheme, if any
-    int colon = pathString.indexOf(':');
-    int slash = pathString.indexOf('/');
-    if (colon > 0 && (slash == colon +1)) {
-      // has a non zero-length scheme
-      scheme = pathString.substring(0, colon);
-      start = colon + 1;
-    }
-
-    // parse uri authority, if any
-    if (pathString.startsWith("//", start) &&
-        (pathString.length()-start > 2)) {
-      start += 2;
-      int nextSlash = pathString.indexOf('/', start);
-      int authEnd = nextSlash > 0 ? nextSlash : pathString.length();
-      authority = pathString.substring(start, authEnd);
-      start = authEnd;
-    }
-    // uri path is the rest of the string. ? or # are not interpreted,
-    // but any occurrence of them will be quoted by the URI ctor.
-    String path = pathString.substring(start, pathString.length());
-
-    // Construct the URI
-    try {
-      return new URI(scheme, authority, path, null, null);
-    } catch (URISyntaxException e) {
-      throw new IllegalArgumentException(e);
-    }
-  }
-
-  /**
-   *
-   * @param uri
-   * @return volumeName
-   * @throws Exception
-   * @throws OzoneClientException when uri is null or invalid volume name
-   */
-  protected String parseVolumeName(String uri) throws Exception{
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    int pathNameCount = path.getNameCount();
-    if (pathNameCount != 1) {
-      String errorMessage;
-      if (pathNameCount < 1) {
-        errorMessage = "Volume name is required to perform volume " +
-            "operations like info, update, create and delete. ";
-      } else {
-        errorMessage = "Invalid volume name. Delimiters (/) not allowed in " +
-            "volume name";
-      }
-      throw new OzoneClientException(errorMessage);
-    }
-
-    return ozoneURI.getPath().replaceAll("^/+", "");
-  }
-
   public boolean isVerbose() {
     return parent.isVerbose();
   }
 
+  public OzoneConfiguration createOzoneConfiguration() {
+    return parent.createOzoneConfiguration();
+  }
+
 }

+ 251 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/OzoneAddress.java

@@ -0,0 +1,251 @@
+/**
+ * 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.web.ozShell;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.rest.OzoneException;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_HTTP_SCHEME;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_RPC_SCHEME;
+import org.apache.http.client.utils.URIBuilder;
+
+/**
+ * Address of an ozone object for ozone shell.
+ */
+public class OzoneAddress {
+
+  private static final String EMPTY_HOST = "___DEFAULT___";
+
+  private URI ozoneURI;
+
+  private String volumeName = "";
+
+  private String bucketName = "";
+
+  private String keyName = "";
+
+  public OzoneAddress() throws OzoneException {
+    this("o3:///");
+  }
+
+  public OzoneAddress(String address)
+      throws OzoneException {
+    if (address == null || address.equals("")) {
+      address = OZONE_RPC_SCHEME + ":///";
+    }
+    this.ozoneURI = parseURI(address);
+    String path = this.ozoneURI.getPath();
+
+    path = path.replaceAll("^/+", "");
+
+    int sep1 = path.indexOf('/');
+    int sep2 = path.indexOf('/', sep1 + 1);
+
+    if (sep1 == -1) {
+      volumeName = path;
+    } else {
+      //we have vol/bucket
+      volumeName = path.substring(0, sep1);
+      if (sep2 == -1) {
+        bucketName = path.substring(sep1 + 1);
+      } else {
+        //we have vol/bucket/key/.../...
+        bucketName = path.substring(sep1 + 1, sep2);
+        keyName = path.substring(sep2 + 1);
+      }
+    }
+
+  }
+
+  public OzoneClient createClient(OzoneConfiguration conf)
+      throws IOException, OzoneClientException {
+    OzoneClient client;
+    String scheme = ozoneURI.getScheme();
+    if (ozoneURI.getScheme() == null || scheme.isEmpty()) {
+      scheme = OZONE_RPC_SCHEME;
+    }
+    if (scheme.equals(OZONE_HTTP_SCHEME)) {
+      if (ozoneURI.getHost() != null && !ozoneURI.getAuthority()
+          .equals(EMPTY_HOST)) {
+        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_RPC_SCHEME)) {
+      if (ozoneURI.getHost() != null && !ozoneURI.getAuthority()
+          .equals(EMPTY_HOST)) {
+        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, unknown protocol scheme: " + scheme);
+    }
+    return client;
+  }
+
+  /**
+   * verifies user provided URI.
+   *
+   * @param uri - UriString
+   * @return URI
+   * @throws URISyntaxException
+   * @throws OzoneException
+   */
+  protected URI parseURI(String uri)
+      throws OzoneException {
+    if ((uri == null) || uri.isEmpty()) {
+      throw new OzoneClientException(
+          "Ozone URI is needed to execute this command.");
+    }
+    URIBuilder uriBuilder = new URIBuilder(stringToUri(uri));
+    if (uriBuilder.getPort() == 0) {
+      uriBuilder.setPort(Shell.DEFAULT_OZONE_PORT);
+    }
+
+    try {
+      return uriBuilder.build();
+    } catch (URISyntaxException e) {
+      throw new OzoneClientException("Invalid URI: " + ozoneURI, e);
+    }
+  }
+
+  /**
+   * Construct a URI from a String with unescaped special characters
+   * that have non-standard semantics. e.g. /, ?, #. A custom parsing
+   * is needed to prevent misbehavior.
+   *
+   * @param pathString The input path in string form
+   * @return URI
+   */
+  private static URI stringToUri(String pathString) {
+    // parse uri components
+    String scheme = null;
+    String authority = null;
+    int start = 0;
+
+    // parse uri scheme, if any
+    int colon = pathString.indexOf(':');
+    int slash = pathString.indexOf('/');
+    if (colon > 0 && (slash == colon + 1)) {
+      // has a non zero-length scheme
+      scheme = pathString.substring(0, colon);
+      start = colon + 1;
+    }
+
+    // parse uri authority, if any
+    if (pathString.startsWith("//", start) &&
+        (pathString.length() - start > 2)) {
+      start += 2;
+      int nextSlash = pathString.indexOf('/', start);
+      int authEnd = nextSlash > 0 ? nextSlash : pathString.length();
+      authority = pathString.substring(start, authEnd);
+      start = authEnd;
+    }
+    // uri path is the rest of the string. ? or # are not interpreted,
+    // but any occurrence of them will be quoted by the URI ctor.
+    String path = pathString.substring(start, pathString.length());
+
+    if (authority == null || authority.equals("")) {
+      authority = EMPTY_HOST;
+    }
+    // Construct the URI
+    try {
+      return new URI(scheme, authority, path, null, null);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  public String getKeyName() {
+    return keyName;
+  }
+
+  public void ensureBucketAddress() throws OzoneClientException {
+    if (keyName.length() > 0) {
+      throw new OzoneClientException(
+          "Invalid bucket name. Delimiters (/) not allowed in bucket name");
+    } else if (volumeName.length() == 0) {
+      throw new OzoneClientException(
+          "Volume name is required.");
+    } else if (bucketName.length() == 0) {
+      throw new OzoneClientException(
+          "Bucket name is required.");
+    }
+  }
+
+  public void ensureKeyAddress() throws OzoneClientException {
+    if (keyName.length() == 0) {
+      throw new OzoneClientException(
+          "Key name is missing.");
+    } else if (volumeName.length() == 0) {
+      throw new OzoneClientException(
+          "Volume name is missing");
+    } else if (bucketName.length() == 0) {
+      throw new OzoneClientException(
+          "Bucket name is missing");
+    }
+  }
+
+  public void ensureVolumeAddress() throws OzoneClientException {
+    if (keyName.length() != 0) {
+      throw new OzoneClientException(
+          "Invalid volume name. Delimiters (/) not allowed in volume name");
+    } else if (volumeName.length() == 0) {
+      throw new OzoneClientException(
+          "Volume name is required");
+    } else if (bucketName.length() != 0) {
+      throw new OzoneClientException(
+          "Invalid volume name. Delimiters (/) not allowed in volume name");
+    }
+  }
+
+  public void ensureRootAddress() throws OzoneClientException {
+    if (keyName.length() != 0 || bucketName.length() != 0
+        || volumeName.length() != 0) {
+      throw new OzoneClientException(
+          "Invalid URI. Volume/bucket/key elements should not been used");
+    }
+  }
+}

+ 6 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/BucketCommands.java

@@ -23,6 +23,7 @@ import java.util.concurrent.Callable;
 import org.apache.hadoop.hdds.cli.GenericParentCommand;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.cli.MissingSubcommandException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 import picocli.CommandLine.Command;
@@ -58,4 +59,9 @@ public class BucketCommands implements GenericParentCommand, Callable<Void> {
   public boolean isVerbose() {
     return shell.isVerbose();
   }
+
+  @Override
+  public OzoneConfiguration createOzoneConfiguration() {
+    return shell.createOzoneConfiguration();
+  }
 }

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

@@ -17,15 +17,12 @@
  */
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
 import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
@@ -48,22 +45,12 @@ public class CreateBucketHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    int pathNameCount = path.getNameCount();
-    if (pathNameCount != 2) {
-      String errorMessage;
-      if (pathNameCount < 2) {
-        errorMessage = "volume and bucket name required in createBucket";
-      } else {
-        errorMessage = "Invalid bucket name. Delimiters (/) not allowed in " +
-            "bucket name";
-      }
-      throw new OzoneClientException(errorMessage);
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureBucketAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
-    String volumeName = path.getName(0).toString();
-    String bucketName = path.getName(1).toString();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
 
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);

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

@@ -18,13 +18,10 @@
 
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 import picocli.CommandLine.Command;
@@ -46,15 +43,12 @@ public class DeleteBucketHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    if (path.getNameCount() < 2) {
-      throw new OzoneClientException(
-          "volume and bucket name required in delete Bucket");
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureBucketAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
-    String volumeName = path.getName(0).toString();
-    String bucketName = path.getName(1).toString();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
 
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);

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

@@ -17,15 +17,12 @@
  */
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
 import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
@@ -47,23 +44,12 @@ public class InfoBucketHandler extends Handler {
    */
   @Override
   public Void call() throws Exception {
-    String volumeName, bucketName;
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    int pathNameCount = path.getNameCount();
-    if (pathNameCount != 2) {
-      String errorMessage;
-      if (pathNameCount < 2) {
-        errorMessage = "volume and bucket name required in infoBucket";
-      } else {
-        errorMessage = "Invalid bucket name. Delimiters (/) not allowed in " +
-            "bucket name";
-      }
-      throw new OzoneClientException(errorMessage);
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureBucketAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
 
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);

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

@@ -18,19 +18,17 @@
 
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 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.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
@@ -69,27 +67,16 @@ public class ListBucketHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    int pathNameCount = path.getNameCount();
-    if (pathNameCount != 1) {
-      String errorMessage;
-      if (pathNameCount < 1) {
-        errorMessage = "volume is required in listBucket";
-      } else {
-        errorMessage = "Invalid volume name. Delimiters (/) not allowed in " +
-            "volume name";
-      }
-      throw new OzoneClientException(errorMessage);
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureVolumeAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
+    String volumeName = address.getVolumeName();
     if (maxBuckets < 1) {
       throw new IllegalArgumentException(
           "the length should be a positive number");
     }
 
-    String volumeName = path.getName(0).toString();
-
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
     }
@@ -114,5 +101,6 @@ public class ListBucketHandler extends Handler {
         JsonUtils.toJsonString(bucketList)));
     return null;
   }
+
 }
 

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

@@ -17,15 +17,10 @@
  */
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
-import org.apache.hadoop.ozone.web.ozShell.Shell;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 
 import picocli.CommandLine.Command;
 import picocli.CommandLine.Parameters;
@@ -38,9 +33,8 @@ import picocli.CommandLine.Parameters;
     description = "Returns the ozone path for S3Bucket")
 public class S3BucketMapping extends Handler {
 
-  @Parameters(arity = "1..1", description = Shell
-      .OZONE_S3BUCKET_URI_DESCRIPTION)
-  private String uri;
+  @Parameters(arity = "1..1", description = "Name of the s3 bucket.")
+  private String s3BucketName;
 
   /**
    * Executes create bucket.
@@ -48,40 +42,13 @@ public class S3BucketMapping extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    int pathNameCount = path.getNameCount();
-    String errorMessage;
-
-    // When just uri is given as http://om:9874, we are getting pathCount
-    // still as 1, as getPath() is returning empty string.
-    // So for safer side check, whether it is an empty string
-    if (pathNameCount == 1) {
-      String s3Bucket = path.getName(0).toString();
-      if (StringUtils.isBlank(s3Bucket)) {
-        errorMessage = "S3Bucket name is required to get volume name and " +
-            "Ozone fs Uri";
-        throw new OzoneClientException(errorMessage);
-      }
-    }
-    if (pathNameCount != 1) {
-      if (pathNameCount < 1) {
-        errorMessage = "S3Bucket name is required to get volume name and " +
-            "Ozone fs Uri";
-      } else {
-        errorMessage = "Invalid S3Bucket name. Delimiters (/) not allowed in " +
-            "S3Bucket name";
-      }
-      throw new OzoneClientException(errorMessage);
-    }
-
-    String s3Bucket = path.getName(0).toString();
-    if (isVerbose()) {
-      System.out.printf("S3Bucket Name : %s%n", s3Bucket);
-    }
+    OzoneAddress ozoneAddress = new OzoneAddress();
+    OzoneClient client = ozoneAddress.createClient(createOzoneConfiguration());
 
-    String mapping = client.getObjectStore().getOzoneBucketMapping(s3Bucket);
-    String volumeName = client.getObjectStore().getOzoneVolumeName(s3Bucket);
+    String mapping =
+        client.getObjectStore().getOzoneBucketMapping(s3BucketName);
+    String volumeName =
+        client.getObjectStore().getOzoneVolumeName(s3BucketName);
 
     if (isVerbose()) {
       System.out.printf("Mapping created for S3Bucket is : %s%n", mapping);
@@ -90,7 +57,7 @@ public class S3BucketMapping extends Handler {
     System.out.printf("Volume name for S3Bucket is : %s%n", volumeName);
 
     String ozoneFsUri = String.format("%s://%s.%s", OzoneConsts
-        .OZONE_URI_SCHEME, s3Bucket, volumeName);
+        .OZONE_URI_SCHEME, s3BucketName, volumeName);
 
     System.out.printf("Ozone FileSystem Uri is : %s%n", ozoneFsUri);
 

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

@@ -17,19 +17,17 @@
  */
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.Arrays;
 import java.util.List;
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
@@ -60,16 +58,12 @@ public class UpdateBucketHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureBucketAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
-    if (path.getNameCount() < 2) {
-      throw new OzoneClientException(
-          "volume and bucket name required in update bucket");
-    }
-
-    String volumeName = path.getName(0).toString();
-    String bucketName = path.getName(1).toString();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
 
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);

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

@@ -18,14 +18,11 @@
 
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
 import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 import picocli.CommandLine.Command;
@@ -47,16 +44,14 @@ public class DeleteKeyHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    if (path.getNameCount() < 3) {
-      throw new OzoneClientException(
-          "volume/bucket/key name required in deleteKey");
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureKeyAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
 
-    String volumeName = path.getName(0).toString();
-    String bucketName = path.getName(1).toString();
-    String keyName = path.getName(2).toString();
 
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);

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

@@ -21,17 +21,18 @@ package org.apache.hadoop.ozone.web.ozShell.keys;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
-import java.net.URI;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 
 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.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 import org.apache.commons.codec.digest.DigestUtils;
@@ -61,16 +62,13 @@ public class GetKeyHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    if (path.getNameCount() < 3) {
-      throw new OzoneClientException(
-          "volume/bucket/key name required in putKey");
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureKeyAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
-    String volumeName = path.getName(0).toString();
-    String bucketName = path.getName(1).toString();
-    String keyName = path.getName(2).toString();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
 
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);

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

@@ -18,17 +18,13 @@
 
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneKeyDetails;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
@@ -49,22 +45,14 @@ public class InfoKeyHandler extends Handler {
    */
   @Override
   public Void call() throws Exception {
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    if (path.getNameCount() < 3) {
-      throw new OzoneClientException(
-          "volume/bucket/key name required in infoKey");
-    }
-
-    String volumeName = path.getName(0).toString();
-    String bucketName = path.getName(1).toString();
 
-    String searchString = volumeName + OzoneConsts.OZONE_URI_DELIMITER +
-        bucketName + OzoneConsts.OZONE_URI_DELIMITER;
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureKeyAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
-    String keyName =
-        uri.substring(uri.indexOf(searchString) +
-                searchString.length());
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
 
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);

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

@@ -23,6 +23,7 @@ import java.util.concurrent.Callable;
 import org.apache.hadoop.hdds.cli.GenericParentCommand;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.cli.MissingSubcommandException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 import picocli.CommandLine.Command;
@@ -57,4 +58,9 @@ public class KeyCommands implements GenericParentCommand, Callable<Void> {
   public boolean isVerbose() {
     return shell.isVerbose();
   }
+
+  @Override
+  public OzoneConfiguration createOzoneConfiguration() {
+    return shell.createOzoneConfiguration();
+  }
 }

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

@@ -18,20 +18,18 @@
 
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
@@ -69,28 +67,18 @@ public class ListKeyHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    int pathNameCount = path.getNameCount();
-    if (pathNameCount != 2) {
-      String errorMessage;
-      if (pathNameCount < 2) {
-        errorMessage = "volume/bucket is required in listKey";
-      } else {
-        errorMessage = "Invalid bucket name. Delimiters (/) not allowed in " +
-            "bucket name";
-      }
-      throw new OzoneClientException(errorMessage);
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureBucketAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
 
     if (maxKeys < 1) {
       throw new IllegalArgumentException(
           "the length should be a positive number");
     }
 
-    String volumeName = path.getName(0).toString();
-    String bucketName = path.getName(1).toString();
-
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("bucket Name : %s%n", bucketName);

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

@@ -20,9 +20,6 @@ package org.apache.hadoop.ozone.web.ozShell.keys;
 
 import java.io.File;
 import java.io.FileInputStream;
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
@@ -30,10 +27,11 @@ import org.apache.hadoop.hdds.client.ReplicationType;
 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.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 import org.apache.commons.codec.digest.DigestUtils;
@@ -71,16 +69,13 @@ public class PutKeyHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    if (path.getNameCount() < 3) {
-      throw new OzoneClientException(
-          "volume/bucket/key name required in putKey");
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureKeyAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
-    String volumeName = path.getName(0).toString();
-    String bucketName = path.getName(1).toString();
-    String keyName = path.getName(2).toString();
+    String volumeName = address.getVolumeName();
+    String bucketName = address.getBucketName();
+    String keyName = address.getKeyName();
 
     if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);

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

@@ -18,19 +18,16 @@
 
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
-import java.net.URI;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClient;
 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.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
-
 import org.apache.hadoop.security.UserGroupInformation;
+
 import picocli.CommandLine.Command;
 import picocli.CommandLine.Option;
 import picocli.CommandLine.Parameters;
@@ -68,21 +65,12 @@ public class CreateVolumeHandler extends Handler {
       userName = UserGroupInformation.getCurrentUser().getUserName();
     }
 
-    URI ozoneURI = verifyURI(uri);
-    Path path = Paths.get(ozoneURI.getPath());
-    int pathNameCount = path.getNameCount();
-    if (pathNameCount != 1) {
-      String errorMessage;
-      if (pathNameCount < 1) {
-        errorMessage = "Volume name is required to create a volume";
-      } else {
-        errorMessage = "Invalid volume name. Delimiters (/) not allowed in " +
-            "volume name";
-      }
-      throw new OzoneClientException(errorMessage);
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureVolumeAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    String volumeName = address.getVolumeName();
 
-    String volumeName = ozoneURI.getPath().replaceAll("^/+", "");
     if (isVerbose()) {
       System.out.printf("Volume name : %s%n", volumeName);
     }

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

@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 import picocli.CommandLine.Command;
@@ -40,7 +42,11 @@ public class DeleteVolumeHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    String volumeName = parseVolumeName(uri);
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureVolumeAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    String volumeName = address.getVolumeName();
 
     if (isVerbose()) {
       System.out.printf("Volume name : %s%n", volumeName);

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

@@ -18,9 +18,11 @@
 
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
@@ -43,7 +45,11 @@ public class InfoVolumeHandler extends Handler{
   @Override
   public Void call() throws Exception {
 
-    String volumeName = parseVolumeName(uri);
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureVolumeAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    String volumeName = address.getVolumeName();
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(

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

@@ -18,23 +18,22 @@
 
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
-import com.google.common.base.Strings;
-import picocli.CommandLine.Command;
-import picocli.CommandLine.Option;
-import picocli.CommandLine.Parameters;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
 
+import org.apache.hadoop.ozone.client.OzoneClient;
 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.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
 
 /**
  * Executes List Volume call.
@@ -72,13 +71,9 @@ public class ListVolumeHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    URI ozoneURI = verifyURI(uri);
-    if (!Strings.isNullOrEmpty(ozoneURI.getPath()) && !ozoneURI.getPath()
-        .equals("/")) {
-      throw new OzoneClientException(
-          "Invalid URI: " + ozoneURI + " . Specified path not used." + ozoneURI
-              .getPath());
-    }
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureRootAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
 
     if (userName == null) {
       userName = System.getProperty("user.name");

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

@@ -23,9 +23,11 @@ import picocli.CommandLine.Option;
 import picocli.CommandLine.Parameters;
 
 import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.OzoneAddress;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
@@ -54,7 +56,11 @@ public class UpdateVolumeHandler extends Handler {
   @Override
   public Void call() throws Exception {
 
-    String volumeName = parseVolumeName(uri);
+    OzoneAddress address = new OzoneAddress(uri);
+    address.ensureVolumeAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    String volumeName = address.getVolumeName();
 
     OzoneVolume volume = client.getObjectStore().getVolume(volumeName);
     if (quota != null && !quota.isEmpty()) {

+ 6 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/VolumeCommands.java

@@ -23,6 +23,7 @@ import java.util.concurrent.Callable;
 import org.apache.hadoop.hdds.cli.GenericParentCommand;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.cli.MissingSubcommandException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
 import picocli.CommandLine.Command;
@@ -58,4 +59,9 @@ public class VolumeCommands implements GenericParentCommand, Callable<Void> {
   public boolean isVerbose() {
     return shell.isVerbose();
   }
+
+  @Override
+  public OzoneConfiguration createOzoneConfiguration() {
+    return shell.createOzoneConfiguration();
+  }
 }

+ 100 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/web/ozShell/TestOzoneAddress.java

@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.web.ozShell;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.ozone.client.rest.OzoneException;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Test ozone URL parsing.
+ */
+@RunWith(Parameterized.class)
+public class TestOzoneAddress {
+
+  @Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][] {
+        {"o3fs://localhost:9878/"},
+        {"o3fs://localhost/"},
+        {"o3fs:///"},
+        {"http://localhost:9878/"},
+        {"http://localhost/"},
+        {"http:///"},
+        {"/"}
+    });
+  }
+
+  private String prefix;
+
+  public TestOzoneAddress(String prefix) {
+    this.prefix = prefix;
+  }
+
+  @Test
+  public void checkUrlTypes() throws OzoneException, IOException {
+    OzoneAddress address;
+
+    address = new OzoneAddress("");
+    address.ensureRootAddress();
+
+    address = new OzoneAddress(prefix + "");
+    address.ensureRootAddress();
+
+    address = new OzoneAddress(prefix + "vol1");
+    address.ensureVolumeAddress();
+    Assert.assertEquals("vol1", address.getVolumeName());
+
+    address = new OzoneAddress(prefix + "vol1/bucket");
+    address.ensureBucketAddress();
+    Assert.assertEquals("vol1", address.getVolumeName());
+    Assert.assertEquals("bucket", address.getBucketName());
+
+    address = new OzoneAddress(prefix + "vol1/bucket/");
+    address.ensureBucketAddress();
+    Assert.assertEquals("vol1", address.getVolumeName());
+    Assert.assertEquals("bucket", address.getBucketName());
+
+    address = new OzoneAddress(prefix + "vol1/bucket/key");
+    address.ensureKeyAddress();
+    Assert.assertEquals("vol1", address.getVolumeName());
+    Assert.assertEquals("bucket", address.getBucketName());
+    Assert.assertEquals("key", address.getKeyName());
+
+    address = new OzoneAddress(prefix + "vol1/bucket/key/");
+    address.ensureKeyAddress();
+    Assert.assertEquals("vol1", address.getVolumeName());
+    Assert.assertEquals("bucket", address.getBucketName());
+    Assert.assertEquals("key/", address.getKeyName());
+
+    address = new OzoneAddress(prefix + "vol1/bucket/key1/key3/key");
+    address.ensureKeyAddress();
+    Assert.assertEquals("vol1", address.getVolumeName());
+    Assert.assertEquals("bucket", address.getBucketName());
+    Assert.assertEquals("key1/key3/key", address.getKeyName());
+  }
+}

+ 21 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/web/ozShell/package-info.java

@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.web.ozShell;
+/**
+ * Tests for ozone shell..
+ */