Procházet zdrojové kódy

YARN-11664. Remove HDFS Binaries/Jars Dependency From Yarn (#6631)

To support YARN deployments in clusters without HDFS
some changes have been made in packaging

* new hadoop-common class org.apache.hadoop.fs.HdfsCommonConstants
* hdfs class org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair moved
  from hdfs-client to hadoop-common
* YARN handlers for DSQuotaExceededException replaced by use of superclass
  ClusterStorageCapacityExceededException.  

Contributed by Syed Shameerur Rahman
Syed Shameerur Rahman před 8 měsíci
rodič
revize
6c01490f14
10 změnil soubory, kde provedl 100 přidání a 18 odebrání
  1. 47 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HdfsCommonConstants.java
  2. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java
  3. 25 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/package-info.java
  4. 7 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenIdentifier.java
  5. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  6. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
  7. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
  8. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java
  9. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
  10. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java

+ 47 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HdfsCommonConstants.java

@@ -0,0 +1,47 @@
+/*
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class contains constants for configuration keys and default values.
+ */
+@InterfaceAudience.LimitedPrivate({"YARN", "HDFS"})
+@InterfaceStability.Evolving
+public final class HdfsCommonConstants {
+
+  /**
+   * Represents the kind of delegation token used for HDFS.
+   * This is a constant string value "HDFS_DELEGATION_TOKEN".
+   */
+  public static final Text HDFS_DELEGATION_KIND =
+      new Text("HDFS_DELEGATION_TOKEN");
+
+  /**
+   * DFS_ADMIN configuration: {@value}.
+   */
+  public static final String DFS_ADMIN = "dfs.cluster.administrators";
+
+  private HdfsCommonConstants() {
+  }
+
+}

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java

@@ -22,13 +22,13 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 
-import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.IOUtils;
 
 /**
- * A little struct class to wrap an InputStream and an OutputStream.
+ * This class wraps an InputStream and an OutputStream.
+ * Both the InputStream and OutputStream are closed on close call.
+ * This class is moved from HDFS module to COMMON module for removing HDFS dependencies from YARN.
  */
-@InterfaceAudience.Private
 public class IOStreamPair implements Closeable {
   public final InputStream in;
   public final OutputStream out;

+ 25 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/package-info.java

@@ -0,0 +1,25 @@
+/**
+ * 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.
+ */
+
+/**
+ * This package provides access to following class.
+ * {@link org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair} class.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.hdfs.protocol.datatransfer;
+
+import org.apache.hadoop.classification.InterfaceAudience;

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenIdentifier.java

@@ -26,6 +26,7 @@ import java.util.Map;
 
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.HdfsCommonConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -41,8 +42,12 @@ import org.apache.hadoop.classification.VisibleForTesting;
 @InterfaceAudience.Private
 public class DelegationTokenIdentifier
     extends AbstractDelegationTokenIdentifier {
-  public static final Text HDFS_DELEGATION_KIND =
-      new Text("HDFS_DELEGATION_TOKEN");
+
+  /**
+   * The value is referenced from {@link HdfsCommonConstants#HDFS_DELEGATION_KIND}.
+   */
+  @Deprecated
+  public static final Text HDFS_DELEGATION_KIND = HdfsCommonConstants.HDFS_DELEGATION_KIND;
 
   @SuppressWarnings("unchecked")
   private static Map<TokenIdentifier, UserGroupInformation> ugiCache =

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.HdfsCommonConstants;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.net.DFSNetworkTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -381,7 +382,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
   public static final String  DFS_NAMENODE_XATTRS_ENABLED_KEY = "dfs.namenode.xattrs.enabled";
   public static final boolean DFS_NAMENODE_XATTRS_ENABLED_DEFAULT = true;
-  public static final String  DFS_ADMIN = "dfs.cluster.administrators";
+  /**
+   * The value is referenced from {@link HdfsCommonConstants#DFS_ADMIN}.
+   */
+  @Deprecated
+  public static final String  DFS_ADMIN = HdfsCommonConstants.DFS_ADMIN;
   public static final String  DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY = "dfs.https.server.keystore.resource";
   public static final String  DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT = "ssl-server.xml";
   public static final String  DFS_SERVER_HTTPS_KEYPASSWORD_KEY = "ssl.server.keystore.keypassword";

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java

@@ -23,8 +23,8 @@ import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Options;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.HdfsCommonConstants;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -230,7 +230,7 @@ public class ServiceMaster extends CompositeService {
     while (iter.hasNext()) {
       Token<? extends TokenIdentifier> token = iter.next();
       if (token.getKind().equals(
-          DelegationTokenIdentifier.HDFS_DELEGATION_KIND)) {
+          HdfsCommonConstants.HDFS_DELEGATION_KIND)) {
         LOG.info("Remove HDFS delegation token {}.", token);
         iter.remove();
       }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java

@@ -31,10 +31,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.HdfsCommonConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.registry.client.api.RegistryConstants;
@@ -1707,12 +1707,12 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
         YarnConfiguration.YARN_ADMIN_ACL,
         YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
     AccessControlList dfsAdminAcl = new AccessControlList(
-        getConfig().get(DFSConfigKeys.DFS_ADMIN, " "));
+        getConfig().get(HdfsCommonConstants.DFS_ADMIN, " "));
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     if (!yarnAdminAcl.isUserAllowed(ugi) && !dfsAdminAcl.isUserAllowed(ugi)) {
       LOG.error("User must be on the {} or {} list to have permission to " +
           "upload AM dependency tarball", YarnConfiguration.YARN_ADMIN_ACL,
-          DFSConfigKeys.DFS_ADMIN);
+          HdfsCommonConstants.DFS_ADMIN);
       return false;
     }
 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java

@@ -20,9 +20,9 @@ package org.apache.hadoop.yarn.service.client;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.HdfsCommonConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.client.cli.ApplicationCLI;
@@ -138,7 +138,7 @@ public class TestServiceCLI {
       basedir.mkdirs();
     }
     yarnAdminNoneAclProp = YarnConfiguration.YARN_ADMIN_ACL + "=none";
-    dfsAdminAclProp = DFSConfigKeys.DFS_ADMIN + "=" +
+    dfsAdminAclProp = HdfsCommonConstants.DFS_ADMIN + "=" +
         UserGroupInformation.getCurrentUser();
     System.setProperty(YarnServiceConstants.PROPERTY_LIB_DIR, basedir
         .getAbsolutePath());

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ClusterStorageCapacityExceededException;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -60,7 +61,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.Writable;
@@ -547,7 +547,7 @@ public class AggregatedLogFormat {
     }
 
     @Override
-    public void close() throws DSQuotaExceededException {
+    public void close() throws ClusterStorageCapacityExceededException {
       try {
         if (writer != null) {
           writer.close();
@@ -557,7 +557,7 @@ public class AggregatedLogFormat {
       } finally {
         try {
           this.fsDataOStream.close();
-        } catch (DSQuotaExceededException e) {
+        } catch (ClusterStorageCapacityExceededException e) {
           LOG.error("Exception in closing {}",
               this.fsDataOStream.getClass(), e);
           throw e;

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java

@@ -38,12 +38,12 @@ import org.apache.commons.math3.util.Pair;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ClusterStorageCapacityExceededException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.HarFs;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
@@ -99,7 +99,7 @@ public class LogAggregationTFileController
     if (this.writer != null) {
       try {
         this.writer.close();
-      } catch (DSQuotaExceededException e) {
+      } catch (ClusterStorageCapacityExceededException e) {
         throw new LogAggregationDFSException(e);
       } finally {
         this.writer = null;