Просмотр исходного кода

HDDS-975. Manage ozone security tokens with ozone shell cli. Contributed by Ajay Kumar.

Ajay Kumar 6 лет назад
Родитель
Сommit
dcbc8b86ed

+ 39 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java

@@ -27,12 +27,15 @@ import java.util.Objects;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
+import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
+import org.apache.hadoop.security.token.Token;
 
 /**
  * ObjectStore class is responsible for the client operations that can be
@@ -378,4 +381,40 @@ public class ObjectStore {
     }
   }
 
+  /**
+   * Get a valid Delegation Token.
+   *
+   * @param renewer the designated renewer for the token
+   * @return Token<OzoneDelegationTokenSelector>
+   * @throws IOException
+   */
+  public Token<OzoneTokenIdentifier> getDelegationToken(Text renewer)
+      throws IOException {
+    return proxy.getDelegationToken(renewer);
+  }
+
+  /**
+   * Renew an existing delegation token.
+   *
+   * @param token delegation token obtained earlier
+   * @return the new expiration time
+   * @throws IOException
+   */
+  public long renewDelegationToken(Token<OzoneTokenIdentifier> token)
+      throws IOException {
+    return proxy.renewDelegationToken(token);
+  }
+
+  /**
+   * Cancel an existing delegation token.
+   *
+   * @param token delegation token
+   * @throws IOException
+   */
+  public void cancelDelegationToken(Token<OzoneTokenIdentifier> token)
+      throws IOException {
+    proxy.cancelDelegationToken(token);
+  }
+
+
 }

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

@@ -18,16 +18,21 @@
 package org.apache.hadoop.ozone.ozShell;
 
 import java.io.ByteArrayOutputStream;
+import java.io.EOFException;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.Random;
 import java.util.UUID;
 import java.util.stream.Collectors;
@@ -65,6 +70,7 @@ 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.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -477,6 +483,27 @@ public class TestOzoneShell {
     }
   }
 
+  /**
+   * Execute command, assert exception message and returns true if error
+   * was thrown.
+   */
+  private void executeWithError(Shell ozoneShell, String[] args,
+      Class exception) {
+    if (Objects.isNull(exception)) {
+      execute(ozoneShell, args);
+    } else {
+      try {
+        execute(ozoneShell, args);
+        fail("Exception is expected from command execution " + Arrays
+            .asList(args));
+      } catch (Exception ex) {
+        LOG.error("Exception: ", ex);
+        assertTrue(ex.getCause().getClass().getCanonicalName()
+            .equals(exception.getCanonicalName()));
+      }
+    }
+  }
+
   /**
    * Execute command, assert exception message and returns true if error
    * was thrown and contains the specified usage string.
@@ -1246,6 +1273,73 @@ public class TestOzoneShell {
     return bucketInfo;
   }
 
+  @Test
+  public void testTokenCommands() throws Exception {
+    String omHostName = cluster.getOzoneManager().getServiceList().stream()
+        .filter(a -> a.getNodeType().equals(HddsProtos.NodeType.OM))
+        .collect(Collectors.toList()).get(0).getHostname();
+
+    String omPort = cluster.getOzoneManager().getRpcPort();
+    String omAdd = "--set=" + OZONE_OM_ADDRESS_KEY + "=" + omHostName
+        + ":" + omPort;
+    List<String[]> shellCommands = new ArrayList<>(4);
+    // Case 1: Execution will fail when security is disabled.
+    shellCommands.add(new String[]{omAdd, "token", "get"});
+    shellCommands.add(new String[]{omAdd, "token", "renew"});
+    shellCommands.add(new String[]{omAdd, "token", "cancel"});
+    shellCommands.add(new String[]{omAdd, "token", "print"});
+    shellCommands.forEach(cmd -> execute(cmd, "Error:Token operations " +
+        "work only"));
+
+    String security = "-D=" + OZONE_SECURITY_ENABLED_KEY + "=true";
+
+    // Case 2: Execution of get token will fail when security is enabled but
+    // OzoneManager is not setup correctly.
+    execute(new String[]{omAdd, security,
+        "token", "get"}, "Error: Get delegation token operation failed.");
+
+    // Clear all commands.
+    shellCommands.clear();
+
+    // Case 3: Execution of renew/cancel/print token will fail as token file
+    // doesn't exist.
+    shellCommands.add(new String[]{omAdd, security, "token", "renew"});
+    shellCommands.add(new String[]{omAdd, security, "token", "cancel"});
+    shellCommands.add(new String[]{omAdd, security, "token", "print"});
+    shellCommands.forEach(cmd -> execute(cmd, "token " +
+        "operation failed as token file:"));
+
+    // Create corrupt token file.
+    File testPath = GenericTestUtils.getTestDir();
+    Files.createDirectories(testPath.toPath());
+    Path tokenFile = Paths.get(testPath.toString(), "token.txt");
+    String question = RandomStringUtils.random(100);
+    Files.write(tokenFile, question.getBytes());
+
+    // Clear all commands.
+    shellCommands.clear();
+    String file = "-t=" + tokenFile.toString();
+
+    // Case 4: Execution of renew/cancel/print token will fail if token file
+    // is corrupt.
+    shellCommands.add(new String[]{omAdd, security, "token", "renew", file});
+    shellCommands.add(new String[]{omAdd, security, "token",
+        "cancel", file});
+    shellCommands.add(new String[]{omAdd, security, "token", "print", file});
+    shellCommands.forEach(cmd -> executeWithError(shell, cmd,
+        EOFException.class));
+  }
+
+  private void execute(String[] cmd, String msg) {
+    // verify the response output
+    execute(shell, cmd);
+    String output = err.toString();
+    assertTrue(output.contains(msg));
+    // reset stream
+    out.reset();
+    err.reset();
+  }
+
   /**
    * Create a temporary file used for putting key.
    * @return the created file's path string

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

@@ -24,6 +24,7 @@ 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.apache.hadoop.ozone.web.ozShell.token.TokenCommands;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -41,6 +42,7 @@ import picocli.CommandLine.Command;
         VolumeCommands.class,
         BucketCommands.class,
         KeyCommands.class,
+        TokenCommands.class,
         S3Commands.class
     },
     versionProvider = HddsVersionProvider.class,

+ 70 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/CancelTokenHandler.java

@@ -0,0 +1,70 @@
+/*
+ * 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.token;
+
+import org.apache.hadoop.ozone.OzoneSecurityUtil;
+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.token.Token;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+/**
+ * Executes cancelDelegationToken api.
+ */
+@Command(name = "cancel",
+    description = "cancel a delegation token.")
+public class CancelTokenHandler extends Handler {
+
+  @CommandLine.Option(names = {"--token", "-t"},
+      description = "file containing encoded token",
+      defaultValue = "/tmp/token.txt",
+      showDefaultValue = CommandLine.Help.Visibility.ALWAYS)
+  private String tokenFile;
+
+  /**
+   * Executes the Client Calls.
+   */
+  @Override
+  public Void call() throws Exception {
+    OzoneAddress address = new OzoneAddress("");
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    if(!OzoneSecurityUtil.isSecurityEnabled(createOzoneConfiguration())) {
+      System.err.println("Error:Token operations work only when security is " +
+          "enabled. To enable security set ozone.security.enabled to true.");
+      return null;
+    }
+
+    if (Files.notExists(Paths.get(tokenFile))) {
+      System.err.println("Error:Cancel token operation failed as token file: "
+          + tokenFile + " containing encoded token doesn't exist.");
+      return null;
+    }
+    Token token = new Token();
+    token.decodeFromUrlString(
+        new String(Files.readAllBytes(Paths.get(tokenFile))));
+    client.getObjectStore().cancelDelegationToken(token);
+    return null;
+  }
+}

+ 77 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/GetTokenHandler.java

@@ -0,0 +1,77 @@
+/*
+ * 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.token;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ozone.OzoneSecurityUtil;
+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.utils.JsonUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+
+import java.util.Objects;
+
+/**
+ * Executes getDelegationToken api.
+ */
+@Command(name = "get",
+    description = "get a delegation token.")
+public class GetTokenHandler extends Handler {
+
+
+
+  @CommandLine.Option(names = {"--renewer", "-r"},
+      description = "Token renewer",
+      showDefaultValue = CommandLine.Help.Visibility.ALWAYS)
+  private String renewer;
+
+  /**
+   * Executes the Client Calls.
+   */
+  @Override
+  public Void call() throws Exception {
+    OzoneAddress address = new OzoneAddress();
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    if(!OzoneSecurityUtil.isSecurityEnabled(createOzoneConfiguration())) {
+      System.err.println("Error:Token operations work only when security is " +
+          "enabled. To enable security set ozone.security.enabled to true.");
+      return null;
+    }
+
+    if(StringUtils.isEmpty(renewer)){
+      renewer = UserGroupInformation.getCurrentUser().getShortUserName();
+    }
+    Token token = client.getObjectStore().getDelegationToken(new Text(renewer));
+    if(Objects.isNull(token)){
+      System.err.println("Error: Get delegation token operation failed. Check" +
+          " OzoneManager logs for more details.");
+      return null;
+    }
+
+    System.out.printf("%s", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+        JsonUtils.toJsonString(token.encodeToUrlString())));
+    return null;
+  }
+}

+ 69 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/PrintTokenHandler.java

@@ -0,0 +1,69 @@
+/*
+ * 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.token;
+
+import org.apache.hadoop.ozone.OzoneSecurityUtil;
+import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.utils.JsonUtils;
+import org.apache.hadoop.security.token.Token;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+/**
+ * Executes getDelegationToken api.
+ */
+@Command(name = "print",
+    description = "print a delegation token.")
+public class PrintTokenHandler extends Handler {
+
+  @CommandLine.Option(names = {"--token", "-t"},
+      description = "file containing encoded token",
+      defaultValue = "/tmp/token.txt",
+      showDefaultValue = CommandLine.Help.Visibility.ALWAYS)
+  private String tokenFile;
+
+  /**
+   * Executes the Client Calls.
+   */
+  @Override
+  public Void call() throws Exception {
+    if (!OzoneSecurityUtil.isSecurityEnabled(createOzoneConfiguration())) {
+      System.err.println("Error:Token operations work only when security is " +
+          "enabled. To enable security set ozone.security.enabled to true.");
+      return null;
+    }
+
+    if (Files.notExists(Paths.get(tokenFile))) {
+      System.err.println("Error: Print token operation failed as token file: "
+          + tokenFile + " containing encoded token doesn't exist.");
+      return null;
+    }
+
+    String encodedToken = new String(Files.readAllBytes(Paths.get(tokenFile)));
+    Token token = new Token();
+    token.decodeFromUrlString(encodedToken);
+
+    System.out.printf("%s", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+        JsonUtils.toJsonString(token.toString())));
+    return null;
+  }
+}

+ 73 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/RenewTokenHandler.java

@@ -0,0 +1,73 @@
+/*
+ * 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.token;
+
+import org.apache.hadoop.ozone.OzoneSecurityUtil;
+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.token.Token;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+/**
+ * Executes renewDelegationToken api.
+ */
+@Command(name = "renew",
+    description = "renew a delegation token.")
+public class RenewTokenHandler extends Handler {
+
+  @CommandLine.Option(names = {"--token", "-t"},
+      description = "file containing encoded token",
+      defaultValue = "/tmp/token.txt",
+      showDefaultValue = CommandLine.Help.Visibility.ALWAYS)
+  private String tokenFile;
+
+  /**
+   * Executes the Client Calls.
+   */
+  @Override
+  public Void call() throws Exception {
+    OzoneAddress address = new OzoneAddress("");
+    OzoneClient client = address.createClient(createOzoneConfiguration());
+
+    if(!OzoneSecurityUtil.isSecurityEnabled(createOzoneConfiguration())) {
+      System.err.println("Error:Token operations work only when security is " +
+          "enabled. To enable security set ozone.security.enabled to true.");
+      return null;
+    }
+
+    if (Files.notExists(Paths.get(tokenFile))) {
+      System.err.println("Error:Renew token operation failed as token file: "
+          + tokenFile + " containing encoded token doesn't exist.");
+      return null;
+    }
+    Token token = new Token();
+    token.decodeFromUrlString(
+        new String(Files.readAllBytes(Paths.get(tokenFile))));
+    long expiryTime = client.getObjectStore().renewDelegationToken(token);
+
+    System.out.printf("Token renewed successfully, expiry time: %s",
+        expiryTime);
+    return null;
+  }
+}

+ 64 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/TokenCommands.java

@@ -0,0 +1,64 @@
+/*
+ * 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.token;
+
+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;
+
+/**
+ * Sub-command to group token related operations.
+ */
+@Command(name = "token",
+    description = "Token specific operations",
+    subcommands = {
+        GetTokenHandler.class,
+        CancelTokenHandler.class,
+        RenewTokenHandler.class,
+        PrintTokenHandler.class
+    },
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class TokenCommands implements GenericParentCommand, Callable<Void> {
+
+  @ParentCommand
+  private Shell shell;
+
+  @Override
+  public Void call() throws Exception {
+    throw new MissingSubcommandException(
+        this.shell.getCmd().getSubcommands().get("token").getUsageMessage());
+  }
+
+  @Override
+  public boolean isVerbose() {
+    return shell.isVerbose();
+  }
+
+  @Override
+  public OzoneConfiguration createOzoneConfiguration() {
+    return shell.createOzoneConfiguration();
+  }
+}

+ 26 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/package-info.java

@@ -0,0 +1,26 @@
+/*
+ * 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.token;
+
+/**
+ Ozone delegation token commands.
+ **/