You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cn...@apache.org on 2016/03/14 18:52:38 UTC

hadoop git commit: HDFS-9926. ozone : Add volume commands to CLI. Contributed by Anu Engineer.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 64add6756 -> 657bc690c


HDFS-9926. ozone : Add volume commands to CLI. Contributed by Anu Engineer.


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

Branch: refs/heads/HDFS-7240
Commit: 657bc690c294b5fcf1c5777d2fa1259faa27621e
Parents: 64add67
Author: Chris Nauroth <cn...@apache.org>
Authored: Mon Mar 14 10:50:54 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Mon Mar 14 10:50:54 2016 -0700

----------------------------------------------------------------------
 .../hadoop/ozone/web/ozShell/Handler.java       |  76 ++++++
 .../apache/hadoop/ozone/web/ozShell/Shell.java  | 238 +++++++++++++++++++
 .../hadoop/ozone/web/ozShell/package-info.java  |  27 +++
 .../web/ozShell/volume/CreateVolumeHandler.java | 102 ++++++++
 .../web/ozShell/volume/DeleteVolumeHandler.java |  81 +++++++
 .../web/ozShell/volume/InfoVolumeHandler.java   |  99 ++++++++
 .../web/ozShell/volume/ListVolumeHandler.java   | 100 ++++++++
 .../web/ozShell/volume/UpdateVolumeHandler.java |  92 +++++++
 8 files changed, 815 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/657bc690/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
new file mode 100644
index 0000000..8132c7a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
@@ -0,0 +1,76 @@
+/*
+ * 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 org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneClient;
+import org.apache.hadoop.ozone.web.client.OzoneClientException;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.http.client.utils.URIBuilder;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Common interface for command handling.
+ */
+public abstract class Handler {
+  protected OzoneClient client;
+
+  /**
+   * Constructs a client object.
+   */
+  public Handler() {
+    client = new OzoneClient();
+  }
+
+  /**
+   * Executes the Client command.
+   *
+   * @param cmd - CommandLine
+   * @throws IOException
+   * @throws OzoneException
+   * @throws URISyntaxException
+   */
+  protected abstract void execute(CommandLine cmd)
+      throws IOException, OzoneException, URISyntaxException;
+
+  /**
+   * verifies user provided URI.
+   *
+   * @param uri - UriString
+   * @return URI
+   * @throws URISyntaxException
+   * @throws OzoneException
+   */
+  protected URI verifyURI(String uri) throws URISyntaxException,
+      OzoneException {
+    if ((uri == null) || uri.isEmpty()) {
+      throw new OzoneClientException(
+          "Ozone URI is needed to execute this command.");
+    }
+    URIBuilder ozoneURI = new URIBuilder(uri);
+
+    if (ozoneURI.getPort() == 0) {
+      ozoneURI.setPort(Shell.DEFAULT_OZONE_PORT);
+    }
+    return ozoneURI.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/657bc690/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
new file mode 100644
index 0000000..fb0cd77
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
@@ -0,0 +1,238 @@
+/*
+ * 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 org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.ozShell.volume.CreateVolumeHandler;
+import org.apache.hadoop.ozone.web.ozShell.volume.DeleteVolumeHandler;
+import org.apache.hadoop.ozone.web.ozShell.volume.InfoVolumeHandler;
+import org.apache.hadoop.ozone.web.ozShell.volume.ListVolumeHandler;
+import org.apache.hadoop.ozone.web.ozShell.volume.UpdateVolumeHandler;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+/**
+ * Ozone user interface commands.
+ *
+ * This class uses dispatch method to make calls
+ * to appropriate handlers that execute the ozone functions.
+ */
+public class Shell extends Configured implements Tool {
+
+  // General options
+  public static final int DEFAULT_OZONE_PORT = 50070;
+  public static final String VERBOSE = "v";
+
+  // volume related command line arguments
+  public static final String RUNAS = "root";
+  public static final String USER = "user";
+  public static final String OWNER = "owner";
+  public static final String QUOTA = "quota";
+  public static final String CREATE_VOLUME = "createVolume";
+  public static final String UPDATE_VOLUME = "updateVolume";
+  public static final String DELETE_VOLUME = "deleteVolume";
+  public static final String LIST_VOLUME = "listVolume";
+  public static final String INFO_VOLUME = "infoVolume";
+
+
+  /**
+   * Execute the command with the given arguments.
+   *
+   * @param args command specific arguments.
+   *
+   * @return exit code.
+   *
+   * @throws Exception
+   */
+  @Override
+  public int run(String[] args) throws Exception {
+    Options opts = getOpts();
+    CommandLine cmd = parseArgs(args, opts);
+    return dispatch(cmd, opts);
+  }
+
+  /**
+   * Construct an ozShell.
+   */
+  public Shell() {
+  }
+
+  /**
+   * returns the Command Line Options.
+   *
+   * @return Options
+   */
+  private Options getOpts() {
+    Options opts = new Options();
+    addVolumeCommands(opts);
+    return opts;
+  }
+
+  /**
+   * This function parses all command line arguments
+   * and returns the appropriate values.
+   *
+   * @param argv - Argv from main
+   *
+   * @return CommandLine
+   */
+  private CommandLine parseArgs(String[] argv, Options opts)
+      throws org.apache.commons.cli.ParseException {
+    BasicParser parser = new BasicParser();
+    return parser.parse(opts, argv);
+  }
+
+
+  /**
+   * All volume related commands are added in this function for the command
+   * parser.
+   *
+   * @param options - Command Options class.
+   */
+  private void addVolumeCommands(Options options) {
+    Option verbose = new Option(VERBOSE, false, "verbose information output.");
+    options.addOption(verbose);
+
+    Option runas = new Option(RUNAS, false, "Run the command as \"hdfs\" user");
+    options.addOption(runas);
+
+    Option userName = new Option(USER, true,
+                                 "Name of the user in volume management " +
+                                     "functions");
+    options.addOption(userName);
+
+    Option quota = new Option(QUOTA, true, "Quota for the volume. E.g. 10TB");
+    options.addOption(quota);
+
+
+    Option createVolume = new Option(CREATE_VOLUME, true, "creates a volume" +
+        "for the specified user.\n \t For example : hdfs oz  -createVolume " +
+        "<volumeURI> -root -user <userName>\n");
+    options.addOption(createVolume);
+
+    Option deleteVolume = new Option(DELETE_VOLUME, true, "deletes a volume" +
+        "if it is empty.\n \t For example : hdfs oz -deleteVolume <volumeURI>" +
+        " -root \n");
+    options.addOption(deleteVolume);
+
+    Option listVolume =
+        new Option(LIST_VOLUME, true, "List the volumes of a given user.\n" +
+            "For example : hdfs oz -listVolume <ozoneURI>" +
+            "-user <username> -root or hdfs oz " +
+            "-listVolume");
+    options.addOption(listVolume);
+
+    Option updateVolume =
+        new Option(UPDATE_VOLUME, true, "updates an existing volume.\n" +
+            "\t For example : hdfs oz " +
+            "-updateVolume <volumeURI> -quota " +
+            "100TB\n");
+    options.addOption(updateVolume);
+
+    Option infoVolume = new Option(INFO_VOLUME, true,
+                                   "returns information about a specific " +
+                                       "volume.");
+    options.addOption(infoVolume);
+  }
+
+  /**
+   * Main for the ozShell Command handling.
+   *
+   * @param argv - System Args Strings[]
+   *
+   * @throws Exception
+   */
+  public static void main(String[] argv) throws Exception {
+    Shell shell = new Shell();
+    Configuration conf = new Configuration();
+    conf.setQuietMode(false);
+    shell.setConf(conf);
+    int res = 0;
+    try {
+      res = ToolRunner.run(shell, argv);
+    } catch (Exception ex) {
+      System.exit(1);
+    }
+    System.exit(res);
+  }
+
+  /**
+   * Dispatches calls to the right command Handler classes.
+   *
+   * @param cmd - CommandLine
+   *
+   * @throws IOException
+   * @throws OzoneException
+   * @throws URISyntaxException
+   */
+  private int dispatch(CommandLine cmd, Options opts)
+      throws IOException, OzoneException, URISyntaxException {
+    Handler handler = null;
+
+    try {
+
+      // volume functions
+      if (cmd.hasOption(Shell.CREATE_VOLUME)) {
+        handler = new CreateVolumeHandler();
+      }
+
+      if (cmd.hasOption(Shell.DELETE_VOLUME)) {
+        handler = new DeleteVolumeHandler();
+      }
+
+      if (cmd.hasOption(Shell.LIST_VOLUME)) {
+        handler = new ListVolumeHandler();
+      }
+
+      if (cmd.hasOption(Shell.UPDATE_VOLUME)) {
+        handler = new UpdateVolumeHandler();
+      }
+
+      if (cmd.hasOption(Shell.INFO_VOLUME)) {
+        handler = new InfoVolumeHandler();
+      }
+
+      if (handler != null) {
+        handler.execute(cmd);
+        return 0;
+      } else {
+        HelpFormatter helpFormatter = new HelpFormatter();
+        helpFormatter
+            .printHelp(80, "hdfs oz -command uri [args]", "Ozone Commands",
+                       opts, "Please correct your command and try again.");
+        return 1;
+
+      }
+    } catch (IOException | OzoneException | URISyntaxException ex) {
+      System.err.printf("Command Failed : %s%n", ex.getMessage());
+      return 1;
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/657bc690/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/package-info.java
new file mode 100644
index 0000000..e33b6e7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/**
+ *  ozShell Class acts as the command line interface to
+ *  the ozone Rest Client.
+ */
+package org.apache.hadoop.ozone.web.ozShell;
+
+/**
+ A simple CLI to work against Ozone.
+ **/

http://git-wip-us.apache.org/repos/asf/hadoop/blob/657bc690/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
new file mode 100644
index 0000000..cd5a0f9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
@@ -0,0 +1,102 @@
+/*
+ * 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.volume;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneClientException;
+import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.Shell;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Executes the create volume call for the shell.
+ */
+public class CreateVolumeHandler extends Handler {
+
+  private String rootName;
+  private String userName;
+  private String volumeName;
+  private String quota;
+
+  /**
+   * Executes the Create Volume.
+   *
+   * @param cmd - CommandLine
+   * @throws IOException
+   * @throws OzoneException
+   * @throws URISyntaxException
+   */
+  @Override
+  protected void execute(CommandLine cmd)
+      throws IOException, OzoneException, URISyntaxException {
+    if (!cmd.hasOption(Shell.CREATE_VOLUME)) {
+      throw new OzoneClientException(
+          "Incorrect call : createVolume is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.CREATE_VOLUME);
+    URI ozoneURI = verifyURI(ozoneURIString);
+    if (ozoneURI.getPath().isEmpty()) {
+      throw new OzoneClientException(
+          "Volume name is required to create a volume");
+    }
+
+    // we need to skip the slash in the URI path
+    // getPath returns /volumeName needs to remove the first slash.
+    volumeName = ozoneURI.getPath().substring(1);
+
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      System.out.printf("Volume name : %s%n", volumeName);
+    }
+    if (cmd.hasOption(Shell.RUNAS)) {
+      rootName = "hdfs";
+    } else {
+      rootName = System.getProperty("user.name");
+    }
+
+    if (!cmd.hasOption(Shell.USER)) {
+      throw new OzoneClientException(
+          "User name is needed in createVolume call.");
+    }
+
+    if (cmd.hasOption(Shell.QUOTA)) {
+      quota = cmd.getOptionValue(Shell.QUOTA);
+    }
+
+    userName = cmd.getOptionValue(Shell.USER);
+    client.setEndPointURI(ozoneURI);
+    client.setUserAuth(rootName);
+
+
+    OzoneVolume vol = client.createVolume(volumeName, userName, quota);
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      ObjectMapper mapper = new ObjectMapper();
+      Object json = mapper.readValue(vol.getJsonString(), Object.class);
+      System.out.printf("%s%n", mapper.writerWithDefaultPrettyPrinter()
+          .writeValueAsString(json));
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/657bc690/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
new file mode 100644
index 0000000..4ca7ede
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
@@ -0,0 +1,81 @@
+/*
+ * 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.volume;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneClientException;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.Shell;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Executes deleteVolume call for the shell.
+ */
+public class DeleteVolumeHandler extends Handler {
+
+  private String volumeName;
+  private String rootName;
+
+  /**
+   * Executes the delete volume call.
+   *
+   * @param cmd - CommandLine
+   * @throws IOException
+   * @throws OzoneException
+   * @throws URISyntaxException
+   */
+  @Override
+  protected void execute(CommandLine cmd)
+      throws IOException, OzoneException, URISyntaxException {
+
+    if (!cmd.hasOption(Shell.DELETE_VOLUME)) {
+      throw new OzoneClientException(
+          "Incorrect call : deleteVolume call is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.DELETE_VOLUME);
+    URI ozoneURI = verifyURI(ozoneURIString);
+    if (ozoneURI.getPath().isEmpty()) {
+      throw new OzoneClientException(
+          "Volume name is required to delete a volume");
+    }
+
+    // we need to skip the slash in the URI path
+    volumeName = ozoneURI.getPath().substring(1);
+
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      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);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/657bc690/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
new file mode 100644
index 0000000..9ce5620
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
@@ -0,0 +1,99 @@
+/*
+ * 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.volume;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneClientException;
+import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.Shell;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Executes volume Info calls.
+ */
+public class InfoVolumeHandler extends Handler{
+
+  private String rootName;
+  private String volumeName;
+  private String userName;
+
+  /**
+   * Executes volume Info.
+   *
+   * @param cmd - CommandLine
+   *
+   * @throws IOException
+   * @throws OzoneException
+   * @throws URISyntaxException
+   */
+  @Override
+  protected void execute(CommandLine cmd)
+      throws IOException, OzoneException, URISyntaxException {
+
+    if (!cmd.hasOption(Shell.INFO_VOLUME)) {
+      throw new OzoneClientException(
+          "Incorrect call : infoVolume is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.INFO_VOLUME);
+    URI ozoneURI = verifyURI(ozoneURIString);
+    if (ozoneURI.getPath().isEmpty()) {
+      throw new OzoneClientException(
+          "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
+    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);
+    ObjectMapper mapper = new ObjectMapper();
+    Object json = mapper.readValue(vol.getJsonString(), Object.class);
+    System.out.printf("%s%n", mapper.writerWithDefaultPrettyPrinter()
+                          .writeValueAsString(json));
+
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/657bc690/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
new file mode 100644
index 0000000..25735b9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.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.volume;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneClientException;
+import org.apache.hadoop.ozone.web.client.OzoneVolume;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.Shell;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+
+/**
+ * Executes List Volume call.
+ */
+public class ListVolumeHandler extends Handler {
+  private String rootName;
+  private String userName;
+
+  /**
+   * Executes the Client Calls.
+   *
+   * @param cmd - CommandLine
+   * @throws IOException
+   * @throws OzoneException
+   * @throws URISyntaxException
+   */
+  @Override
+  protected void execute(CommandLine cmd)
+      throws IOException, OzoneException, URISyntaxException {
+
+    if (!cmd.hasOption(Shell.LIST_VOLUME)) {
+      throw new OzoneClientException(
+          "Incorrect call : listVolume is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.LIST_VOLUME);
+    URI ozoneURI = verifyURI(ozoneURIString);
+
+    if (cmd.hasOption(Shell.RUNAS)) {
+      rootName = "hdfs";
+    }
+
+    if (!cmd.hasOption(Shell.USER)) {
+      throw new OzoneClientException(
+          "User name is needed in listVolume call.");
+    }
+
+    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);
+    }
+
+    List<OzoneVolume> volumes = client.listVolumes(userName);
+    if (volumes != null) {
+      if (cmd.hasOption(Shell.VERBOSE)) {
+        System.out.printf("Found : %d volumes for user : %s %n", volumes.size(),
+            userName);
+      }
+      ObjectMapper mapper = new ObjectMapper();
+
+      for (OzoneVolume vol : volumes) {
+        Object json = mapper.readValue(vol.getJsonString(), Object.class);
+        System.out.printf("%s%n", mapper.writerWithDefaultPrettyPrinter()
+            .writeValueAsString(json));
+      }
+
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/657bc690/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
new file mode 100644
index 0000000..a325c57
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
@@ -0,0 +1,92 @@
+/*
+ * 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.volume;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneClientException;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.Shell;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Executes update volume calls.
+ */
+public class UpdateVolumeHandler extends Handler {
+  private String rootName;
+  private String ownerName;
+  private String volumeName;
+  private String quota;
+
+  /**
+   * Executes update volume calls.
+   *
+   * @param cmd - CommandLine
+   * @throws IOException
+   * @throws OzoneException
+   * @throws URISyntaxException
+   */
+  @Override
+  protected void execute(CommandLine cmd)
+      throws IOException, OzoneException, URISyntaxException {
+    if (!cmd.hasOption(Shell.UPDATE_VOLUME)) {
+      throw new OzoneClientException(
+          "Incorrect call : updateVolume is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.UPDATE_VOLUME);
+    URI ozoneURI = verifyURI(ozoneURIString);
+    if (ozoneURI.getPath().isEmpty()) {
+      throw new OzoneClientException(
+          "Volume name is required to update a volume");
+    }
+
+    // we need to skip the slash in the URI path
+    volumeName = ozoneURI.getPath().substring(1);
+
+    if (cmd.hasOption(Shell.RUNAS)) {
+      rootName = "hdfs";
+    } else {
+      rootName = System.getProperty("user.name");
+    }
+
+    if (cmd.hasOption(Shell.QUOTA)) {
+      quota = cmd.getOptionValue(Shell.QUOTA);
+    }
+
+    if (cmd.hasOption(Shell.OWNER)) {
+      ownerName = cmd.getOptionValue(Shell.OWNER);
+    }
+
+    client.setEndPointURI(ozoneURI);
+    client.setUserAuth(rootName);
+
+    if (quota != null && !quota.isEmpty()) {
+      client.setVolumeQuota(volumeName, quota);
+    }
+
+    if (ownerName != null && !ownerName.isEmpty()) {
+      client.setVolumeOwner(volumeName, ownerName);
+    }
+
+  }
+}