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/16 05:30:21 UTC

hadoop git commit: HDFS-9961. Ozone: Add buckets commands to CLI. Contributed by Anu Engineer.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 4683d68ee -> 4143f0362


HDFS-9961. Ozone: Add buckets 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/4143f036
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4143f036
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4143f036

Branch: refs/heads/HDFS-7240
Commit: 4143f0362fcabc2e7256c624db9f65ad9d060f27
Parents: 4683d68
Author: Chris Nauroth <cn...@apache.org>
Authored: Tue Mar 15 21:27:53 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Tue Mar 15 21:27:53 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ozone/web/ozShell/Shell.java  | 68 +++++++++++++-
 .../web/ozShell/bucket/CreateBucketHandler.java | 99 ++++++++++++++++++++
 .../web/ozShell/bucket/DeleteBucketHandler.java | 87 +++++++++++++++++
 .../web/ozShell/bucket/InfoBucketHandler.java   | 97 +++++++++++++++++++
 .../web/ozShell/bucket/ListBucketHandler.java   | 99 ++++++++++++++++++++
 5 files changed, 446 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4143f036/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
index fb0cd77..a0d3d23 100644
--- 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
@@ -31,6 +31,10 @@ 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.ozone.web.ozShell.bucket.CreateBucketHandler;
+import org.apache.hadoop.ozone.web.ozShell.bucket.DeleteBucketHandler;
+import org.apache.hadoop.ozone.web.ozShell.bucket.InfoBucketHandler;
+import org.apache.hadoop.ozone.web.ozShell.bucket.ListBucketHandler;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -60,6 +64,15 @@ public class Shell extends Configured implements Tool {
   public static final String LIST_VOLUME = "listVolume";
   public static final String INFO_VOLUME = "infoVolume";
 
+  // bucket related command line arguments
+  public static final String CREATE_BUCKET = "createBucket";
+  public static final String UPDATE_BUCKET = "updateBucket";
+  public static final String DELETE_BUCKET = "deleteBucket";
+  public static final String LIST_BUCKET = "listBucket";
+  public static final String INFO_BUCKET = "infoBucket";
+  public static final String ADD_ACLS = "addAcl";
+  public static final String REMOVE_ACLS = "removeAcl";
+
 
   /**
    * Execute the command with the given arguments.
@@ -91,6 +104,7 @@ public class Shell extends Configured implements Tool {
   private Options getOpts() {
     Options opts = new Options();
     addVolumeCommands(opts);
+    addBucketCommands(opts);
     return opts;
   }
 
@@ -162,6 +176,34 @@ public class Shell extends Configured implements Tool {
   }
 
   /**
+   * All bucket related commands for ozone.
+   *
+   * @param opts - Options
+   */
+  private void addBucketCommands(Options opts) {
+    Option createBucket = new Option(CREATE_BUCKET, true,
+        "creates a bucket in a given volume.\n" +
+            "\t For example : hdfs oz " +
+            "-createBucket " +
+            "<volumeName/bucketName>");
+    opts.addOption(createBucket);
+
+    Option infoBucket =
+        new Option(INFO_BUCKET, true, "returns information about a bucket.");
+    opts.addOption(infoBucket);
+
+    Option deleteBucket =
+        new Option(DELETE_BUCKET, true, "deletes an empty bucket.");
+    opts.addOption(deleteBucket);
+
+    Option listBucket =
+        new Option(LIST_BUCKET, true, "Lists the buckets in a volume.");
+    opts.addOption(listBucket);
+
+  }
+
+
+  /**
    * Main for the ozShell Command handling.
    *
    * @param argv - System Args Strings[]
@@ -194,6 +236,7 @@ public class Shell extends Configured implements Tool {
   private int dispatch(CommandLine cmd, Options opts)
       throws IOException, OzoneException, URISyntaxException {
     Handler handler = null;
+    final int eightyColumn = 80;
 
     try {
 
@@ -218,16 +261,33 @@ public class Shell extends Configured implements Tool {
         handler = new InfoVolumeHandler();
       }
 
+      // bucket functions
+      if (cmd.hasOption(Shell.CREATE_BUCKET)) {
+        handler = new CreateBucketHandler();
+      }
+
+      if (cmd.hasOption(Shell.DELETE_BUCKET)) {
+        handler = new DeleteBucketHandler();
+      }
+
+      if (cmd.hasOption(Shell.INFO_BUCKET)) {
+        handler = new InfoBucketHandler();
+      }
+
+      if (cmd.hasOption(Shell.LIST_BUCKET)) {
+        handler = new ListBucketHandler();
+      }
+
+
       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.");
+        helpFormatter.printHelp(eightyColumn, "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());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4143f036/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java
new file mode 100644
index 0000000..4800499
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.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.bucket;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneBucket;
+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.nio.file.Path;
+import java.nio.file.Paths;
+
+/**
+ * create bucket handler.
+ */
+public class CreateBucketHandler extends Handler {
+
+  private String volumeName;
+  private String bucketName;
+  private String rootName;
+
+  /**
+   * Executes create bucket.
+   *
+   * @param cmd - CommandLine
+   *
+   * @throws IOException
+   * @throws OzoneException
+   * @throws URISyntaxException
+   */
+  @Override
+  protected void execute(CommandLine cmd)
+      throws IOException, OzoneException, URISyntaxException {
+    if (!cmd.hasOption(Shell.CREATE_BUCKET)) {
+      throw new OzoneClientException(
+          "Incorrect call : createBucket is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.CREATE_BUCKET);
+    URI ozoneURI = verifyURI(ozoneURIString);
+    Path path = Paths.get(ozoneURI.getPath());
+    if (path.getNameCount() < 2) {
+      throw new OzoneClientException(
+          "volume and bucket name required in createBucket");
+    }
+
+    volumeName = path.getName(0).toString();
+    bucketName = path.getName(1).toString();
+
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      System.out.printf("Volume Name : %s%n", volumeName);
+      System.out.printf("Bucket Name : %s%n", bucketName);
+    }
+
+    if (cmd.hasOption(Shell.RUNAS)) {
+      rootName = "hdfs";
+    } else {
+      rootName = System.getProperty("user.name");
+    }
+
+
+    client.setEndPointURI(ozoneURI);
+    client.setUserAuth(rootName);
+
+
+    OzoneVolume vol = client.getVolume(volumeName);
+    OzoneBucket bucket = vol.createBucket(bucketName);
+
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      ObjectMapper mapper = new ObjectMapper();
+      Object json = mapper.readValue(bucket.getBucketInfo().toJsonString(),
+          Object.class);
+      System.out.printf("%s%n", mapper.writerWithDefaultPrettyPrinter()
+          .writeValueAsString(json));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4143f036/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
new file mode 100644
index 0000000..46c9493
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
@@ -0,0 +1,87 @@
+/*
+ * 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.bucket;
+
+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 java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+/**
+ * Delete bucket Handler.
+ */
+public class DeleteBucketHandler extends Handler {
+  private String volumeName;
+  private String bucketName;
+  private String rootName;
+
+  /**
+   * 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.DELETE_BUCKET)) {
+      throw new OzoneClientException(
+          "Incorrect call : deleteBucket is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.DELETE_BUCKET);
+    URI ozoneURI = verifyURI(ozoneURIString);
+    Path path = Paths.get(ozoneURI.getPath());
+    if (path.getNameCount() < 2) {
+      throw new OzoneClientException(
+          "volume and bucket name required in delete Bucket");
+    }
+
+    volumeName = path.getName(0).toString();
+    bucketName = path.getName(1).toString();
+
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      System.out.printf("Volume Name : %s%n", volumeName);
+      System.out.printf("Bucket Name : %s%n", bucketName);
+    }
+
+    if (cmd.hasOption(Shell.RUNAS)) {
+      rootName = "hdfs";
+    } else {
+      rootName = System.getProperty("user.name");
+    }
+
+    client.setEndPointURI(ozoneURI);
+    client.setUserAuth(rootName);
+
+    OzoneVolume vol = client.getVolume(volumeName);
+    vol.deleteBucket(bucketName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4143f036/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
new file mode 100644
index 0000000..6694768
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
@@ -0,0 +1,97 @@
+/*
+ * 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.bucket;
+
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneBucket;
+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.nio.file.Path;
+import java.nio.file.Paths;
+
+/**
+ * Executes Info bucket.
+ */
+public class InfoBucketHandler extends Handler {
+  private String volumeName;
+  private String bucketName;
+  private String rootName;
+
+  /**
+   * 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.INFO_BUCKET)) {
+      throw new OzoneClientException(
+          "Incorrect call : infoBucket is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.INFO_BUCKET);
+    URI ozoneURI = verifyURI(ozoneURIString);
+    Path path = Paths.get(ozoneURI.getPath());
+
+    if (path.getNameCount() < 2) {
+      throw new OzoneClientException(
+          "volume and bucket name required in delete Bucket");
+    }
+
+    volumeName = path.getName(0).toString();
+    bucketName = path.getName(1).toString();
+
+    if (cmd.hasOption(Shell.VERBOSE)) {
+      System.out.printf("Volume Name : %s%n", volumeName);
+      System.out.printf("Bucket Name : %s%n", bucketName);
+    }
+
+    if (cmd.hasOption(Shell.RUNAS)) {
+      rootName = "hdfs";
+    } else {
+      rootName = System.getProperty("user.name");
+    }
+
+    client.setEndPointURI(ozoneURI);
+    client.setUserAuth(rootName);
+
+    OzoneVolume vol = client.getVolume(volumeName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+
+    ObjectMapper mapper = new ObjectMapper();
+    Object json =
+        mapper.readValue(bucket.getBucketInfo().toJsonString(), Object.class);
+    System.out.printf("%s%n", mapper.writerWithDefaultPrettyPrinter()
+        .writeValueAsString(json));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4143f036/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
new file mode 100644
index 0000000..af844a0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.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.bucket;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.ozone.web.client.OzoneBucket;
+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.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+
+/**
+ * Executes List Bucket.
+ */
+public class ListBucketHandler extends Handler {
+  private String volumeName;
+  private String rootName;
+
+  /**
+   * 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_BUCKET)) {
+      throw new OzoneClientException("Incorrect call : listBucket is missing");
+    }
+
+    String ozoneURIString = cmd.getOptionValue(Shell.LIST_BUCKET);
+    URI ozoneURI = verifyURI(ozoneURIString);
+    Path path = Paths.get(ozoneURI.getPath());
+    if (path.getNameCount() < 1) {
+      throw new OzoneClientException("volume is required in listBucket");
+    }
+
+    volumeName = path.getName(0).toString();
+
+
+    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);
+
+
+    OzoneVolume vol = client.getVolume(volumeName);
+    List<OzoneBucket> bucketList = vol.listBuckets();
+
+    ObjectMapper mapper = new ObjectMapper();
+
+
+    for (OzoneBucket bucket : bucketList) {
+      Object json =
+          mapper.readValue(bucket.getBucketInfo().toJsonString(), Object.class);
+      System.out.printf("%s%n", mapper.writerWithDefaultPrettyPrinter()
+          .writeValueAsString(json));
+    }
+  }
+}
+