Browse Source

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 10 months ago
parent
commit
6c01490f14
10 changed files with 100 additions and 18 deletions
  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;