소스 검색

HDDS-1246. Add ozone delegation token utility subcmd for Ozone CLI. Contributed by Xiaoyu Yao. (#594)

Xiaoyu Yao 6 년 전
부모
커밋
5cfb88a225

+ 12 - 2
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

@@ -140,7 +140,6 @@ public class RpcClient implements ClientProtocol {
         OMConfigKeys.OZONE_OM_GROUP_RIGHTS_DEFAULT);
     this.ozoneManagerClient = new OzoneManagerProtocolClientSideTranslatorPB(
         this.conf, clientId.toString(), ugi);
-
     long scmVersion =
         RPC.getProtocolVersion(StorageContainerLocationProtocolPB.class);
     InetSocketAddress scmAddress = getScmAddressForClient();
@@ -449,7 +448,18 @@ public class RpcClient implements ClientProtocol {
   @Override
   public Token<OzoneTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
-    return ozoneManagerClient.getDelegationToken(renewer);
+
+    Token<OzoneTokenIdentifier> token =
+        ozoneManagerClient.getDelegationToken(renewer);
+    if (token != null) {
+      Text dtService =
+          getOMProxyProvider().getProxy().getDelegationTokenService();
+      token.setService(dtService);
+      LOG.debug("Created token {}", token);
+    } else {
+      LOG.debug("Cannot get ozone delegation token from {}", renewer);
+    }
+    return token;
   }
 
   /**

+ 5 - 0
hadoop-ozone/common/src/main/bin/ozone

@@ -50,6 +50,7 @@ function hadoop_usage
   hadoop_add_subcommand "sh" client "command line interface for object store operations"
   hadoop_add_subcommand "s3" client "command line interface for s3 related operations"
   hadoop_add_subcommand "version" client "print the version"
+  hadoop_add_subcommand "dtutil" client "operations related to delegation tokens"
 
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" false
 }
@@ -163,6 +164,10 @@ function ozonecmd_case
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.genconf.GenerateOzoneRequiredConfigurations
       OZONE_RUN_ARTIFACT_NAME="hadoop-ozone-tools"
     ;;
+    dtutil)
+      HADOOP_CLASSNAME=org.apache.hadoop.security.token.DtUtilShell
+      OZONE_RUN_ARTIFACT_NAME="hadoop-ozone-tools"
+    ;;
     *)
       HADOOP_CLASSNAME="${subcmd}"
       if ! hadoop_validate_classname "${HADOOP_CLASSNAME}"; then

+ 11 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/ha/OMFailoverProxyProvider.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.om.ha;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.FailoverProxyProvider;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -29,6 +30,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -84,16 +86,23 @@ public class OMFailoverProxyProvider implements
   public final class OMProxyInfo
       extends FailoverProxyProvider.ProxyInfo<OzoneManagerProtocolPB> {
     private InetSocketAddress address;
+    private Text dtService;
 
     OMProxyInfo(OzoneManagerProtocolPB proxy, String proxyInfoStr,
+        Text dtService,
         InetSocketAddress addr) {
       super(proxy, proxyInfoStr);
       this.address = addr;
+      this.dtService = dtService;
     }
 
     public InetSocketAddress getAddress() {
       return address;
     }
+
+    public Text getDelegationTokenService() {
+      return dtService;
+    }
   }
 
   private void loadOMClientConfigs(Configuration config) throws IOException {
@@ -125,11 +134,12 @@ public class OMFailoverProxyProvider implements
 
         // Add the OM client proxy info to list of proxies
         if (addr != null) {
+          Text dtService = SecurityUtil.buildTokenService(addr);
           StringBuilder proxyInfo = new StringBuilder()
               .append(nodeId).append("(")
               .append(NetUtils.getHostPortString(addr)).append(")");
           OMProxyInfo omProxyInfo = new OMProxyInfo(null,
-              proxyInfo.toString(), addr);
+              proxyInfo.toString(), dtService, addr);
 
           // For a non-HA OM setup, nodeId might be null. If so, we assign it
           // a dummy value

+ 3 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java

@@ -42,13 +42,16 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.ozone.security.OzoneDelegationTokenSelector;
 import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.TokenInfo;
 
 /**
  * Protocol to talk to OM.
  */
 @KerberosInfo(
     serverPrincipal = OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY)
+@TokenInfo(OzoneDelegationTokenSelector.class)
 public interface OzoneManagerProtocol
     extends OzoneManagerSecurityProtocol, Closeable {
 

+ 3 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSelector.java

@@ -44,7 +44,9 @@ public class OzoneDelegationTokenSelector
   public Token selectToken(Text service,
       Collection<Token<? extends TokenIdentifier>> tokens) {
     LOG.trace("Getting token for service {}", service);
-    return super.selectToken(service, tokens);
+    Token token = super.selectToken(service, tokens);
+    LOG.debug("Got tokens: {} for service {}", token, service);
+    return token;
   }
 
 }

+ 84 - 0
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/O3fsDtFetcher.java

@@ -0,0 +1,84 @@
+/**
+ * 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.fs.ozone;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.DtFetcher;
+import org.apache.hadoop.security.token.Token;
+
+
+/**
+ * A DT fetcher for OzoneFileSystem.
+ * It is only needed for the `hadoop dtutil` command.
+ */
+public class O3fsDtFetcher implements DtFetcher {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(O3fsDtFetcher.class);
+
+  private static final String SERVICE_NAME = OzoneConsts.OZONE_URI_SCHEME;
+
+  private static final String FETCH_FAILED =
+      "Fetch ozone delegation token failed";
+
+  /**
+   * Returns the service name for O3fs, which is also a valid URL prefix.
+   */
+  public Text getServiceName() {
+    return new Text(SERVICE_NAME);
+  }
+
+  public boolean isTokenRequired() {
+    return UserGroupInformation.isSecurityEnabled();
+  }
+
+  /**
+   *  Returns Token object via FileSystem, null if bad argument.
+   *  @param conf - a Configuration object used with FileSystem.get()
+   *  @param creds - a Credentials object to which token(s) will be added
+   *  @param renewer  - the renewer to send with the token request
+   *  @param url  - the URL to which the request is sent
+   *  @return a Token, or null if fetch fails.
+   */
+  public Token<?> addDelegationTokens(Configuration conf, Credentials creds,
+      String renewer, String url) throws Exception {
+    if (!url.startsWith(getServiceName().toString())) {
+      url = getServiceName().toString() + "://" + url;
+    }
+    LOG.debug("addDelegationTokens from {} renewer {}.", url, renewer);
+    FileSystem fs = FileSystem.get(URI.create(url), conf);
+    Token<?> token = fs.getDelegationToken(renewer);
+    if (token == null) {
+      LOG.error(FETCH_FAILED);
+      throw new IOException(FETCH_FAILED);
+    }
+    creds.addToken(token.getService(), token);
+    return token;
+  }
+}

+ 19 - 0
hadoop-ozone/ozonefs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.DtFetcher

@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+org.apache.hadoop.fs.ozone.O3fsDtFetcher

+ 17 - 0
hadoop-ozone/ozonefs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier

@@ -0,0 +1,17 @@
+# 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.
+
+org.apache.hadoop.ozone.security.OzoneTokenIdentifier
+org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier

+ 19 - 0
hadoop-ozone/ozonefs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer

@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl$Renewer