فهرست منبع

HDDS-960. Add cli command option for getS3Secret. Contributed by Dinesh Chitlangia.

Bharat Viswanadham 6 سال پیش
والد
کامیت
c0683ed253

+ 41 - 0
hadoop-hdds/docs/content/S3Commands.md

@@ -0,0 +1,41 @@
+---
+title: S3 Commands
+menu:
+   main:
+      parent: Client
+      weight: 3
+---
+<!---
+  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.
+-->
+
+Ozone shell supports the following S3 commands.
+
+  * [getsecret](#get secret)
+
+### Get Secret
+
+User should get the kerberos ticket before using this option.
+
+
+{{< highlight bash >}}
+ozone sh s3 getkey
+{{< /highlight >}}
+Prints the AWS_SECRET_ACCESS_KEY and AWS_ACCESS_KEY_ID for the current user.
+
+
+You can try out these commands from the docker instance of the [Alpha
+Cluster](runningviadocker.html).

+ 1 - 4
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/S3SecretValue.java

@@ -76,9 +76,6 @@ public class S3SecretValue {
 
   @Override
   public String toString() {
-    return "S3SecretValue{" +
-        "kerberosID='" + kerberosID + '\'' +
-        ", awsSecret='" + awsSecret + '\'' +
-        '}';
+    return "awsAccessKey=" + awsAccessKey + "\nawsSecret=" + awsSecret;
   }
 }

+ 40 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java

@@ -1172,6 +1172,46 @@ public class TestOzoneShell {
     executeWithError(shell, args, "S3_BUCKET_NOT_FOUND");
   }
 
+  @Test
+  public void testS3Secret() throws Exception {
+    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;
+
+    err.reset();
+    String outputFirstAttempt;
+    String outputSecondAttempt;
+
+    //First attempt: If secrets are not found in database, they will be created
+    String[] args = new String[] {setOmAddress, "s3", "getsecret"};
+    execute(shell, args);
+    outputFirstAttempt = out.toString();
+    //Extracting awsAccessKey & awsSecret value from output
+    String[] output = outputFirstAttempt.split("\n");
+    String awsAccessKey = output[0].split("=")[1];
+    String awsSecret = output[1].split("=")[1];
+    assertTrue((awsAccessKey != null && awsAccessKey.length() > 0) &&
+            (awsSecret != null && awsSecret.length() > 0));
+
+    out.reset();
+
+    //Second attempt: Since secrets were created in previous attempt, it
+    // should return the same value
+    args = new String[] {setOmAddress, "s3", "getsecret"};
+    execute(shell, args);
+    outputSecondAttempt = out.toString();
+
+    //verifying if secrets from both attempts are same
+    assertTrue(outputFirstAttempt.equals(outputSecondAttempt));
+  }
+
   private void createS3Bucket(String userName, String s3Bucket) {
     try {
       client.createS3Bucket("ozone", s3Bucket);

+ 3 - 2
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.ozone.web.ozShell.bucket.BucketCommands;
 import org.apache.hadoop.ozone.web.ozShell.keys.KeyCommands;
+import org.apache.hadoop.ozone.web.ozShell.s3.S3Commands;
 import org.apache.hadoop.ozone.web.ozShell.volume.VolumeCommands;
 
 import org.slf4j.Logger;
@@ -39,13 +40,13 @@ import picocli.CommandLine.Command;
     subcommands = {
         VolumeCommands.class,
         BucketCommands.class,
-        KeyCommands.class
+        KeyCommands.class,
+        S3Commands.class
     },
     versionProvider = HddsVersionProvider.class,
     mixinStandardHelpOptions = true)
 public class Shell extends GenericCli {
 
-
   private static final Logger LOG = LoggerFactory.getLogger(Shell.class);
 
   public static final String OZONE_URI_DESCRIPTION = "Ozone URI could start "

+ 49 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/s3/GetS3SecretHandler.java

@@ -0,0 +1,49 @@
+/*
+ * 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.s3;
+
+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.security.UserGroupInformation;
+import picocli.CommandLine.Command;
+
+/**
+ * Executes getsecret calls.
+ */
+@Command(name = "getsecret",
+    description = "returns s3 secret for current user")
+public class GetS3SecretHandler extends Handler {
+
+  /**
+   * Executes getS3Secret.
+   */
+  @Override
+  public Void call() throws Exception {
+    OzoneClient client =
+        new OzoneAddress().createClient(createOzoneConfiguration());
+
+    System.out.println(
+        client.getObjectStore().getS3Secret(
+        UserGroupInformation.getCurrentUser().getUserName()
+        ).toString()
+    );
+
+    return null;
+  }
+}

+ 60 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/s3/S3Commands.java

@@ -0,0 +1,60 @@
+/*
+ * 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.s3;
+
+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;
+import picocli.CommandLine.ParentCommand;
+
+import java.util.concurrent.Callable;
+
+/**
+ * Subcommand to group s3 related operations.
+ */
+@Command(name = "s3",
+    description = "S3 specific operations",
+    subcommands = {
+        GetS3SecretHandler.class
+    },
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class S3Commands implements GenericParentCommand, Callable<Void> {
+
+  @ParentCommand
+  private Shell shell;
+
+  @Override
+  public Void call() throws Exception {
+    throw new MissingSubcommandException(
+        this.shell.getCmd().getSubcommands().get("s3").getUsageMessage());
+  }
+
+  @Override
+  public boolean isVerbose() {
+    return shell.isVerbose();
+  }
+
+  @Override
+  public OzoneConfiguration createOzoneConfiguration() {
+    return shell.createOzoneConfiguration();
+  }
+}

+ 21 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/s3/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
+ *
+ *      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.
+ */
+/**
+ * S3 commands for Ozone.
+ */
+package org.apache.hadoop.ozone.web.ozShell.s3;