You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by xy...@apache.org on 2018/05/07 20:35:26 UTC

[01/35] hadoop git commit: YARN-7799. Improved YARN service jar file handling. Contributed by Billie Rinaldi

Repository: hadoop
Updated Branches:
  refs/heads/HDDS-4 9e2cfb2d3 -> a3a1552c3


YARN-7799. Improved YARN service jar file handling.
           Contributed by Billie Rinaldi


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/24eeea8b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/24eeea8b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/24eeea8b

Branch: refs/heads/HDDS-4
Commit: 24eeea8b18749e02ea05b03eb18e3cf0455077c5
Parents: 9e2cfb2
Author: Eric Yang <ey...@apache.org>
Authored: Tue May 1 16:46:34 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Tue May 1 16:46:34 2018 -0400

----------------------------------------------------------------------
 .../hadoop-yarn-services-core/pom.xml           |   5 +
 .../yarn/service/client/ServiceClient.java      |  95 +++++++++++----
 .../yarn/service/utils/CoreFileSystem.java      |   7 ++
 .../yarn/service/client/TestServiceCLI.java     | 118 +++++++++++++++++--
 .../markdown/yarn-service/Configurations.md     |   2 +-
 5 files changed, 195 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/24eeea8b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml
index 3ce8876..7efe8bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml
@@ -173,6 +173,11 @@
     </dependency>
 
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24eeea8b/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 8dd5342..67306d2 100644
--- a/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
+++ b/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
@@ -28,7 +28,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 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;
@@ -37,8 +39,8 @@ import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
@@ -896,13 +898,13 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
 
   protected Path addJarResource(String serviceName,
       Map<String, LocalResource> localResources)
-      throws IOException, SliderException {
+      throws IOException, YarnException {
     Path libPath = fs.buildClusterDirPath(serviceName);
     ProviderUtils
         .addProviderJar(localResources, ServiceMaster.class, SERVICE_CORE_JAR, fs,
             libPath, "lib", false);
     Path dependencyLibTarGzip = fs.getDependencyTarGzip();
-    if (fs.isFile(dependencyLibTarGzip)) {
+    if (actionDependency(null, false) == EXIT_SUCCESS) {
       LOG.info("Loading lib tar from " + dependencyLibTarGzip);
       fs.submitTarGzipAndUpdate(localResources);
     } else {
@@ -1223,18 +1225,18 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     return actionDependency(destinationFolder, true);
   }
 
-  public int actionDependency(String destinationFolder, boolean overwrite)
-      throws IOException, YarnException {
+  public int actionDependency(String destinationFolder, boolean overwrite) {
     String currentUser = RegistryUtils.currentUser();
     LOG.info("Running command as user {}", currentUser);
 
+    Path dependencyLibTarGzip;
     if (destinationFolder == null) {
-      destinationFolder = String.format(YarnServiceConstants.DEPENDENCY_DIR,
-          VersionInfo.getVersion());
+      dependencyLibTarGzip = fs.getDependencyTarGzip();
+    } else {
+      dependencyLibTarGzip = new Path(destinationFolder,
+          YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME
+              + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT);
     }
-    Path dependencyLibTarGzip = new Path(destinationFolder,
-        YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME
-            + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT);
 
     // Check if dependency has already been uploaded, in which case log
     // appropriately and exit success (unless overwrite has been requested)
@@ -1247,24 +1249,71 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
 
     String[] libDirs = ServiceUtils.getLibDirs();
     if (libDirs.length > 0) {
-      File tempLibTarGzipFile = File.createTempFile(
-          YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME + "_",
-          YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT);
-      // copy all jars
-      tarGzipFolder(libDirs, tempLibTarGzipFile, createJarFilter());
-
-      LOG.info("Version Info: " + VersionInfo.getBuildVersion());
-      fs.copyLocalFileToHdfs(tempLibTarGzipFile, dependencyLibTarGzip,
-          new FsPermission(YarnServiceConstants.DEPENDENCY_DIR_PERMISSIONS));
-      LOG.info("To let apps use this tarball, in yarn-site set config property "
-          + "{} to {}", YarnServiceConf.DEPENDENCY_TARBALL_PATH,
-          dependencyLibTarGzip);
-      return EXIT_SUCCESS;
+      File tempLibTarGzipFile = null;
+      try {
+        if (!checkPermissions(dependencyLibTarGzip)) {
+          return EXIT_UNAUTHORIZED;
+        }
+
+        tempLibTarGzipFile = File.createTempFile(
+            YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME + "_",
+            YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT);
+        // copy all jars
+        tarGzipFolder(libDirs, tempLibTarGzipFile, createJarFilter());
+
+        fs.copyLocalFileToHdfs(tempLibTarGzipFile, dependencyLibTarGzip,
+            new FsPermission(YarnServiceConstants.DEPENDENCY_DIR_PERMISSIONS));
+        LOG.info("To let apps use this tarball, in yarn-site set config " +
+                "property {} to {}", YarnServiceConf.DEPENDENCY_TARBALL_PATH,
+            dependencyLibTarGzip);
+        return EXIT_SUCCESS;
+      } catch (IOException e) {
+        LOG.error("Got exception creating tarball and uploading to HDFS", e);
+        return EXIT_EXCEPTION_THROWN;
+      } finally {
+        if (tempLibTarGzipFile != null) {
+          if (!tempLibTarGzipFile.delete()) {
+            LOG.warn("Failed to delete tmp file {}", tempLibTarGzipFile);
+          }
+        }
+      }
     } else {
       return EXIT_FALSE;
     }
   }
 
+  private boolean checkPermissions(Path dependencyLibTarGzip) throws
+      IOException {
+    AccessControlList yarnAdminAcl = new AccessControlList(getConfig().get(
+        YarnConfiguration.YARN_ADMIN_ACL,
+        YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
+    AccessControlList dfsAdminAcl = new AccessControlList(
+        getConfig().get(DFSConfigKeys.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);
+      return false;
+    }
+
+    Path parent = dependencyLibTarGzip.getParent();
+    while (parent != null) {
+      if (fs.getFileSystem().exists(parent)) {
+        FsPermission perm = fs.getFileSystem().getFileStatus(parent)
+            .getPermission();
+        if (!perm.getOtherAction().implies(FsAction.READ_EXECUTE)) {
+          LOG.error("Parent directory {} of {} tarball location {} does not " +
+              "have world read/execute permission", parent, YarnServiceConf
+              .DEPENDENCY_TARBALL_PATH, dependencyLibTarGzip);
+          return false;
+        }
+      }
+      parent = parent.getParent();
+    }
+    return true;
+  }
+
   protected ClientAMProtocol createAMProxy(String serviceName,
       ApplicationReport appReport) throws IOException, YarnException {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24eeea8b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/CoreFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/CoreFileSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/CoreFileSystem.java
index 5c2bac6..50b22e5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/CoreFileSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/CoreFileSystem.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -363,6 +364,12 @@ public class CoreFileSystem {
     if (configuredDependencyTarballPath != null) {
       dependencyLibTarGzip = new Path(configuredDependencyTarballPath);
     }
+    if (dependencyLibTarGzip == null) {
+      dependencyLibTarGzip = new Path(String.format(YarnServiceConstants
+          .DEPENDENCY_DIR, VersionInfo.getVersion()),
+          YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME
+              + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT);
+    }
     return dependencyLibTarGzip;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24eeea8b/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 7290962..c40a39d 100644
--- a/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
+++ b/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,6 +20,10 @@ package org.apache.hadoop.yarn.service.client;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+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;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -27,12 +31,15 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.api.records.Component;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.conf.ExampleAppJson;
+import org.apache.hadoop.yarn.service.conf.YarnServiceConstants;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
 import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,21 +47,33 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.util.Arrays;
 import java.util.List;
 
 import static org.apache.hadoop.yarn.client.api.AppAdminClient.YARN_APP_ADMIN_CLIENT_PREFIX;
+import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.DEPENDENCY_TARBALL_PATH;
 import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.YARN_SERVICE_BASE_PATH;
+import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.EXIT_SUCCESS;
+import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.EXIT_UNAUTHORIZED;
 import static org.mockito.Mockito.spy;
 
 public class TestServiceCLI {
   private static final Logger LOG = LoggerFactory.getLogger(TestServiceCLI
       .class);
 
+  @Rule
+  public TemporaryFolder tmpFolder = new TemporaryFolder();
+
   private Configuration conf = new YarnConfiguration();
-  private File basedir;
   private SliderFileSystem fs;
-  private String basedirProp;
   private ApplicationCLI cli;
+  private File basedir;
+  private String basedirProp;
+  private File dependencyTarGzBaseDir;
+  private Path dependencyTarGz;
+  private String dependencyTarGzProp;
+  private String yarnAdminNoneAclProp;
+  private String dfsAdminAclProp;
 
   private void createCLI() {
     cli = new ApplicationCLI();
@@ -67,12 +86,17 @@ public class TestServiceCLI {
     cli.setConf(conf);
   }
 
+  private int runCLI(String[] args) throws Exception {
+    LOG.info("running CLI: yarn {}", Arrays.asList(args));
+    return ToolRunner.run(cli, ApplicationCLI.preProcessArgs(args));
+  }
+
   private void buildApp(String serviceName, String appDef) throws Throwable {
     String[] args = {"app",
         "-D", basedirProp, "-save", serviceName,
         ExampleAppJson.resourceName(appDef),
         "-appTypes", DUMMY_APP_TYPE};
-    ToolRunner.run(cli, ApplicationCLI.preProcessArgs(args));
+    Assert.assertEquals(EXIT_SUCCESS, runCLI(args));
   }
 
   private void buildApp(String serviceName, String appDef,
@@ -83,7 +107,13 @@ public class TestServiceCLI {
         "-appTypes", DUMMY_APP_TYPE,
         "-updateLifetime", lifetime,
         "-changeQueue", queue};
-    ToolRunner.run(cli, ApplicationCLI.preProcessArgs(args));
+    Assert.assertEquals(EXIT_SUCCESS, runCLI(args));
+  }
+
+  private static Path getDependencyTarGz(File dir) {
+    return new Path(new File(dir, YarnServiceConstants
+        .DEPENDENCY_TAR_GZ_FILE_NAME + YarnServiceConstants
+        .DEPENDENCY_TAR_GZ_FILE_EXT).getAbsolutePath());
   }
 
   @Before
@@ -91,12 +121,22 @@ public class TestServiceCLI {
     basedir = new File("target", "apps");
     basedirProp = YARN_SERVICE_BASE_PATH + "=" + basedir.getAbsolutePath();
     conf.set(YARN_SERVICE_BASE_PATH, basedir.getAbsolutePath());
+    dependencyTarGzBaseDir = tmpFolder.getRoot();
+    dependencyTarGz = getDependencyTarGz(dependencyTarGzBaseDir);
+    dependencyTarGzProp = DEPENDENCY_TARBALL_PATH + "=" + dependencyTarGz
+        .toString();
+    conf.set(DEPENDENCY_TARBALL_PATH, dependencyTarGz.toString());
     fs = new SliderFileSystem(conf);
     if (basedir.exists()) {
       FileUtils.deleteDirectory(basedir);
     } else {
       basedir.mkdirs();
     }
+    yarnAdminNoneAclProp = YarnConfiguration.YARN_ADMIN_ACL + "=none";
+    dfsAdminAclProp = DFSConfigKeys.DFS_ADMIN + "=" +
+        UserGroupInformation.getCurrentUser();
+    System.setProperty(YarnServiceConstants.PROPERTY_LIB_DIR, basedir
+        .getAbsolutePath());
     createCLI();
   }
 
@@ -108,7 +148,7 @@ public class TestServiceCLI {
     cli.stop();
   }
 
-  @Test
+  @Test (timeout = 180000)
   public void testFlexComponents() throws Throwable {
     // currently can only test building apps, since that is the only
     // operation that doesn't require an RM
@@ -122,7 +162,7 @@ public class TestServiceCLI {
     checkApp(serviceName, "master", 1L, 1000L, "qname");
   }
 
-  @Test
+  @Test (timeout = 180000)
   public void testInitiateServiceUpgrade() throws Exception {
     String[] args = {"app", "-upgrade", "app-1",
         "-initiate", ExampleAppJson.resourceName(ExampleAppJson.APP_JSON),
@@ -131,7 +171,7 @@ public class TestServiceCLI {
     Assert.assertEquals(result, 0);
   }
 
-  @Test
+  @Test (timeout = 180000)
   public void testInitiateAutoFinalizeServiceUpgrade() throws Exception {
     String[] args =  {"app", "-upgrade", "app-1",
         "-initiate", ExampleAppJson.resourceName(ExampleAppJson.APP_JSON),
@@ -141,7 +181,7 @@ public class TestServiceCLI {
     Assert.assertEquals(result, 0);
   }
 
-  @Test
+  @Test (timeout = 180000)
   public void testUpgradeInstances() throws Exception {
     conf.set(YARN_APP_ADMIN_CLIENT_PREFIX + DUMMY_APP_TYPE,
         DummyServiceClient.class.getName());
@@ -153,6 +193,68 @@ public class TestServiceCLI {
     Assert.assertEquals(result, 0);
   }
 
+  @Test (timeout = 180000)
+  public void testEnableFastLaunch() throws Exception {
+    fs.getFileSystem().create(new Path(basedir.getAbsolutePath(), "test.jar"))
+        .close();
+
+    Path defaultPath = new Path(dependencyTarGz.toString());
+    Assert.assertFalse("Dependency tarball should not exist before the test",
+        fs.isFile(defaultPath));
+    String[] args = {"app", "-D", dependencyTarGzProp, "-enableFastLaunch",
+        "-appTypes", DUMMY_APP_TYPE};
+    Assert.assertEquals(EXIT_SUCCESS, runCLI(args));
+    Assert.assertTrue("Dependency tarball did not exist after the test",
+        fs.isFile(defaultPath));
+
+    File secondBaseDir = new File(dependencyTarGzBaseDir, "2");
+    Path secondTarGz = getDependencyTarGz(secondBaseDir);
+    Assert.assertFalse("Dependency tarball should not exist before the test",
+        fs.isFile(secondTarGz));
+    String[] args2 = {"app", "-D", yarnAdminNoneAclProp, "-D",
+        dfsAdminAclProp, "-D", dependencyTarGzProp, "-enableFastLaunch",
+        secondBaseDir.getAbsolutePath(), "-appTypes", DUMMY_APP_TYPE};
+    Assert.assertEquals(EXIT_SUCCESS, runCLI(args2));
+    Assert.assertTrue("Dependency tarball did not exist after the test",
+        fs.isFile(secondTarGz));
+  }
+
+  @Test (timeout = 180000)
+  public void testEnableFastLaunchUserPermissions() throws Exception {
+    String[] args = {"app", "-D", yarnAdminNoneAclProp, "-D",
+        dependencyTarGzProp, "-enableFastLaunch", "-appTypes", DUMMY_APP_TYPE};
+    Assert.assertEquals(EXIT_UNAUTHORIZED, runCLI(args));
+  }
+
+  @Test (timeout = 180000)
+  public void testEnableFastLaunchFilePermissions() throws Exception {
+    File badDir = new File(dependencyTarGzBaseDir, "bad");
+    badDir.mkdir();
+    fs.getFileSystem().setPermission(new Path(badDir.getAbsolutePath()),
+        new FsPermission("751"));
+
+    String[] args = {"app", "-D", dependencyTarGzProp, "-enableFastLaunch",
+        badDir.getAbsolutePath(), "-appTypes", DUMMY_APP_TYPE};
+    Assert.assertEquals(EXIT_UNAUTHORIZED, runCLI(args));
+
+    badDir = new File(badDir, "child");
+    badDir.mkdir();
+    fs.getFileSystem().setPermission(new Path(badDir.getAbsolutePath()),
+        new FsPermission("755"));
+
+    String[] args2 = {"app", "-D", dependencyTarGzProp, "-enableFastLaunch",
+        badDir.getAbsolutePath(), "-appTypes", DUMMY_APP_TYPE};
+    Assert.assertEquals(EXIT_UNAUTHORIZED, runCLI(args2));
+
+    badDir = new File(dependencyTarGzBaseDir, "badx");
+    badDir.mkdir();
+    fs.getFileSystem().setPermission(new Path(badDir.getAbsolutePath()),
+        new FsPermission("754"));
+
+    String[] args3 = {"app", "-D", dependencyTarGzProp, "-enableFastLaunch",
+        badDir.getAbsolutePath(), "-appTypes", DUMMY_APP_TYPE};
+    Assert.assertEquals(EXIT_UNAUTHORIZED, runCLI(args3));
+  }
 
   private void checkApp(String serviceName, String compName, long count, Long
       lifetime, String queue) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24eeea8b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md
index 75186dc..524cfb9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/Configurations.md
@@ -100,7 +100,7 @@ System-wide service AM properties can only be configured in the cluster `yarn-si
 
 | System-Level Config Name | Description |
 | ------------ | ------------- |
-|yarn.service.framework.path | HDFS parent directory where the service AM dependency tarball can be found.|
+|yarn.service.framework.path | HDFS path of the service AM dependency tarball. When no file exists at this location, AM dependencies will be uploaded by the RM the first time a service is started or launched. If the RM user does not have permission to upload the file to this location or the location is not world readable, the AM dependency jars will be uploaded each time a service is started or launched. If unspecified, value will be assumed to be /yarn-services/${hadoop.version}/service-dep.tar.gz.|
 |yarn.service.base.path | HDFS parent directory where service artifacts will be stored (default ${user_home_dir}/.yarn/).
 |yarn.service.client-am.retry.max-wait-ms | Max retry time in milliseconds for the service client to talk to the service AM (default 900000, i.e. 15 minutes).|
 |yarn.service.client-am.retry-interval-ms | Retry interval in milliseconds for the service client to talk to the service AM (default 2000, i.e. 2 seconds).|


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/35] hadoop git commit: Revert "HADOOP-14445. Delegation tokens are not shared between KMS instances. Contributed by Xiao Chen and Rushabh S Shah."

Posted by xy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
index 9fb8969..d709ba8 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
@@ -1,4 +1,3 @@
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -32,36 +31,27 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersi
 import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.KMSDelegationToken;
-import org.apache.hadoop.crypto.key.kms.KMSTokenRenewer;
 import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
-import org.apache.hadoop.crypto.key.kms.TestLoadBalancingKMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.ValueQueue;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.MultipleIOException;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
-import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.Whitebox;
-import org.apache.hadoop.util.KMSUtil;
-import org.apache.hadoop.util.KMSUtilFaultInjector;
 import org.apache.hadoop.util.Time;
 import org.apache.http.client.utils.URIBuilder;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -81,6 +71,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.Writer;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.SocketTimeoutException;
 import java.net.URI;
@@ -105,10 +96,6 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.KMS_CLIENT_COPY_LEGACY_TOKEN_KEY;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
-import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_KIND;
-import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_LEGACY_KIND;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -126,20 +113,6 @@ public class TestKMS {
 
   private SSLFactory sslFactory;
 
-  private final KMSUtilFaultInjector oldInjector =
-      KMSUtilFaultInjector.get();
-
-  // Injector to create providers with different ports. Can only happen in tests
-  private final KMSUtilFaultInjector testInjector =
-      new KMSUtilFaultInjector() {
-        @Override
-        public KeyProvider createKeyProviderForTests(String value,
-            Configuration conf) throws IOException {
-          return TestLoadBalancingKMSClientProvider
-              .createKeyProviderForTests(value, conf);
-        }
-      };
-
   // Keep track of all key providers created during a test case, so they can be
   // closed at test tearDown.
   private List<KeyProvider> providersCreated = new LinkedList<>();
@@ -149,12 +122,7 @@ public class TestKMS {
 
   @Before
   public void setUp() throws Exception {
-    GenericTestUtils.setLogLevel(KMSClientProvider.LOG, Level.TRACE);
-    GenericTestUtils
-        .setLogLevel(DelegationTokenAuthenticationHandler.LOG, Level.TRACE);
-    GenericTestUtils
-        .setLogLevel(DelegationTokenAuthenticator.LOG, Level.TRACE);
-    GenericTestUtils.setLogLevel(KMSUtil.LOG, Level.TRACE);
+    setUpMiniKdc();
     // resetting kerberos security
     Configuration conf = new Configuration();
     UserGroupInformation.setConfiguration(conf);
@@ -173,78 +141,24 @@ public class TestKMS {
   }
 
   public static abstract class KMSCallable<T> implements Callable<T> {
-    private List<URL> kmsUrl;
+    private URL kmsUrl;
 
     protected URL getKMSUrl() {
-      return kmsUrl.get(0);
-    }
-
-    protected URL[] getKMSHAUrl() {
-      URL[] urls = new URL[kmsUrl.size()];
-      return kmsUrl.toArray(urls);
-    }
-
-    protected void addKMSUrl(URL url) {
-      if (kmsUrl == null) {
-        kmsUrl = new ArrayList<URL>();
-      }
-      kmsUrl.add(url);
-    }
-
-    /*
-     * The format of the returned value will be
-     * kms://http:kms1.example1.com:port1,kms://http:kms2.example2.com:port2
-     */
-    protected String generateLoadBalancingKeyProviderUriString() {
-      if (kmsUrl == null || kmsUrl.size() == 0) {
-        return null;
-      }
-      StringBuffer sb = new StringBuffer();
-
-      for (int i = 0; i < kmsUrl.size(); i++) {
-        sb.append(KMSClientProvider.SCHEME_NAME + "://" +
-            kmsUrl.get(0).getProtocol() + "@");
-        URL url = kmsUrl.get(i);
-        sb.append(url.getAuthority());
-        if (url.getPath() != null) {
-          sb.append(url.getPath());
-        }
-        if (i < kmsUrl.size() - 1) {
-          sb.append(",");
-        }
-      }
-      return sb.toString();
+      return kmsUrl;
     }
   }
 
   protected KeyProvider createProvider(URI uri, Configuration conf)
       throws IOException {
     final KeyProvider ret = new LoadBalancingKMSClientProvider(
-        new KMSClientProvider[] {new KMSClientProvider(uri, conf, uri)}, conf);
+        new KMSClientProvider[] {new KMSClientProvider(uri, conf)}, conf);
     providersCreated.add(ret);
     return ret;
   }
 
-  /**
-   * create a LoadBalancingKMSClientProvider from an array of URIs.
-   * @param uris an array of KMS URIs
-   * @param conf configuration object
-   * @return a LoadBalancingKMSClientProvider object
-   * @throws IOException
-   */
-  protected LoadBalancingKMSClientProvider createHAProvider(URI[] uris,
-      Configuration conf, String originalUri) throws IOException {
-    KMSClientProvider[] providers = new KMSClientProvider[uris.length];
-    for (int i = 0; i < providers.length; i++) {
-      providers[i] =
-          new KMSClientProvider(uris[i], conf, URI.create(originalUri));
-    }
-    return new LoadBalancingKMSClientProvider(providers, conf);
-  }
-
   private KMSClientProvider createKMSClientProvider(URI uri, Configuration conf)
       throws IOException {
-    final KMSClientProvider ret = new KMSClientProvider(uri, conf, uri);
+    final KMSClientProvider ret = new KMSClientProvider(uri, conf);
     providersCreated.add(ret);
     return ret;
   }
@@ -256,33 +170,22 @@ public class TestKMS {
 
   protected <T> T runServer(int port, String keystore, String password, File confDir,
       KMSCallable<T> callable) throws Exception {
-    return runServer(new int[] {port}, keystore, password, confDir, callable);
-  }
-
-  protected <T> T runServer(int[] ports, String keystore, String password,
-      File confDir, KMSCallable<T> callable) throws Exception {
     MiniKMS.Builder miniKMSBuilder = new MiniKMS.Builder().setKmsConfDir(confDir)
         .setLog4jConfFile("log4j.properties");
     if (keystore != null) {
       miniKMSBuilder.setSslConf(new File(keystore), password);
     }
-    final List<MiniKMS> kmsList = new ArrayList<>();
-    for (int i=0; i< ports.length; i++) {
-      if (ports[i] > 0) {
-        miniKMSBuilder.setPort(ports[i]);
-      }
-      MiniKMS miniKMS = miniKMSBuilder.build();
-      kmsList.add(miniKMS);
-      miniKMS.start();
-      LOG.info("Test KMS running at: " + miniKMS.getKMSUrl());
-      callable.addKMSUrl(miniKMS.getKMSUrl());
+    if (port > 0) {
+      miniKMSBuilder.setPort(port);
     }
+    MiniKMS miniKMS = miniKMSBuilder.build();
+    miniKMS.start();
     try {
+      System.out.println("Test KMS running at: " + miniKMS.getKMSUrl());
+      callable.kmsUrl = miniKMS.getKMSUrl();
       return callable.call();
     } finally {
-      for (MiniKMS miniKMS: kmsList) {
-        miniKMS.stop();
-      }
+      miniKMS.stop();
     }
   }
 
@@ -337,13 +240,6 @@ public class TestKMS {
     return new URI("kms://" + str);
   }
 
-  public static URI[] createKMSHAUri(URL[] kmsUrls) throws Exception {
-    URI[] uris = new URI[kmsUrls.length];
-    for (int i = 0; i < kmsUrls.length; i++) {
-      uris[i] = createKMSUri(kmsUrls[i]);
-    }
-    return uris;
-  }
 
   private static class KerberosConfiguration
       extends javax.security.auth.login.Configuration {
@@ -419,17 +315,19 @@ public class TestKMS {
         principals.toArray(new String[principals.size()]));
   }
 
-  @BeforeClass
-  public static void setUpMiniKdc() throws Exception {
+  private void setUpMiniKdc() throws Exception {
     Properties kdcConf = MiniKdc.createConf();
     setUpMiniKdc(kdcConf);
   }
 
   @After
   public void tearDown() throws Exception {
+    if (kdc != null) {
+      kdc.stop();
+      kdc = null;
+    }
     UserGroupInformation.setShouldRenewImmediatelyForTests(false);
     UserGroupInformation.reset();
-    KMSUtilFaultInjector.set(oldInjector);
     if (!providersCreated.isEmpty()) {
       final MultipleIOException.Builder b = new MultipleIOException.Builder();
       for (KeyProvider kp : providersCreated) {
@@ -447,14 +345,6 @@ public class TestKMS {
     }
   }
 
-  @AfterClass
-  public static void shutdownMiniKdc() {
-    if (kdc != null) {
-      kdc.stop();
-      kdc = null;
-    }
-  }
-
   private <T> T doAs(String user, final PrivilegedExceptionAction<T> action)
       throws Exception {
     UserGroupInformation.loginUserFromKeytab(user, keytab.getAbsolutePath());
@@ -611,10 +501,8 @@ public class TestKMS {
                 Token<?>[] tokens =
                     ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)kp)
                     .addDelegationTokens("myuser", new Credentials());
-                assertEquals(2, tokens.length);
-                assertEquals(KMSDelegationToken.TOKEN_KIND,
-                    tokens[0].getKind());
-                kp.close();
+                Assert.assertEquals(1, tokens.length);
+                Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
                 return null;
               }
             });
@@ -630,9 +518,8 @@ public class TestKMS {
           Token<?>[] tokens =
               ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)kp)
               .addDelegationTokens("myuser", new Credentials());
-          assertEquals(2, tokens.length);
-          assertEquals(KMSDelegationToken.TOKEN_KIND, tokens[0].getKind());
-          kp.close();
+          Assert.assertEquals(1, tokens.length);
+          Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
         }
         return null;
       }
@@ -2124,6 +2011,7 @@ public class TestKMS {
             return null;
           }
         });
+
         nonKerberosUgi.addCredentials(credentials);
 
         try {
@@ -2179,17 +2067,6 @@ public class TestKMS {
     testDelegationTokensOps(true, true);
   }
 
-  private Text getTokenService(KeyProvider provider) {
-    assertTrue("KeyProvider should be an instance of KMSClientProvider",
-        (provider instanceof LoadBalancingKMSClientProvider));
-    assertEquals("Num client providers should be 1", 1,
-        ((LoadBalancingKMSClientProvider)provider).getProviders().length);
-    Text tokenService =
-        (((LoadBalancingKMSClientProvider)provider).getProviders()[0])
-        .getDelegationTokenService();
-    return tokenService;
-  }
-
   private void testDelegationTokensOps(final boolean ssl, final boolean kerb)
       throws Exception {
     final File confDir = getTestDir();
@@ -2221,16 +2098,11 @@ public class TestKMS {
         final URI uri = createKMSUri(getKMSUrl());
         clientConf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
             createKMSUri(getKMSUrl()).toString());
-        clientConf.setBoolean(KMS_CLIENT_COPY_LEGACY_TOKEN_KEY, false);
 
         doAs("client", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
             KeyProvider kp = createProvider(uri, clientConf);
-            // Unset the conf value for key provider path just to be sure that
-            // the key provider created for renew and cancel token is from
-            // token service field.
-            clientConf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
             // test delegation token retrieval
             KeyProviderDelegationTokenExtension kpdte =
                 KeyProviderDelegationTokenExtension.
@@ -2238,10 +2110,13 @@ public class TestKMS {
             final Credentials credentials = new Credentials();
             final Token<?>[] tokens =
                 kpdte.addDelegationTokens("client1", credentials);
-            Text tokenService = getTokenService(kp);
-            assertEquals(1, credentials.getAllTokens().size());
-            assertEquals(TOKEN_KIND,
-                credentials.getToken(tokenService).getKind());
+            Assert.assertEquals(1, credentials.getAllTokens().size());
+            InetSocketAddress kmsAddr =
+                new InetSocketAddress(getKMSUrl().getHost(),
+                    getKMSUrl().getPort());
+            Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
+                credentials.getToken(SecurityUtil.buildTokenService(kmsAddr)).
+                    getKind());
 
             // Test non-renewer user cannot renew.
             for (Token<?> token : tokens) {
@@ -2368,11 +2243,12 @@ public class TestKMS {
         final URI uri = createKMSUri(getKMSUrl());
         clientConf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
             createKMSUri(getKMSUrl()).toString());
-        clientConf.setBoolean(KMS_CLIENT_COPY_LEGACY_TOKEN_KEY, false);
         final KeyProvider kp = createProvider(uri, clientConf);
         final KeyProviderDelegationTokenExtension kpdte =
             KeyProviderDelegationTokenExtension.
                 createKeyProviderDelegationTokenExtension(kp);
+        final InetSocketAddress kmsAddr =
+            new InetSocketAddress(getKMSUrl().getHost(), getKMSUrl().getPort());
 
         // Job 1 (e.g. YARN log aggregation job), with user DT.
         final Collection<Token<?>> job1Token = new HashSet<>();
@@ -2382,17 +2258,16 @@ public class TestKMS {
             // Get a DT and use it.
             final Credentials credentials = new Credentials();
             kpdte.addDelegationTokens("client", credentials);
-            Text tokenService = getTokenService(kp);
             Assert.assertEquals(1, credentials.getAllTokens().size());
-
+            Assert.assertEquals(KMSDelegationToken.TOKEN_KIND, credentials.
+                getToken(SecurityUtil.buildTokenService(kmsAddr)).getKind());
             UserGroupInformation.getCurrentUser().addCredentials(credentials);
             LOG.info("Added kms dt to credentials: {}", UserGroupInformation.
                 getCurrentUser().getCredentials().getAllTokens());
-            final Token<?> token =
+            Token<?> token =
                 UserGroupInformation.getCurrentUser().getCredentials()
-                    .getToken(tokenService);
-            assertNotNull(token);
-            assertEquals(TOKEN_KIND, token.getKind());
+                    .getToken(SecurityUtil.buildTokenService(kmsAddr));
+            Assert.assertNotNull(token);
             job1Token.add(token);
 
             // Decode the token to get max time.
@@ -2427,16 +2302,17 @@ public class TestKMS {
             // Get a new DT, but don't use it yet.
             final Credentials newCreds = new Credentials();
             kpdte.addDelegationTokens("client", newCreds);
-            assertEquals(1, newCreds.getAllTokens().size());
-            final Text tokenService = getTokenService(kp);
-            assertEquals(TOKEN_KIND,
-                newCreds.getToken(tokenService).getKind());
+            Assert.assertEquals(1, newCreds.getAllTokens().size());
+            Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
+                newCreds.getToken(SecurityUtil.buildTokenService(kmsAddr)).
+                    getKind());
 
             // Using job 1's DT should fail.
             final Credentials oldCreds = new Credentials();
             for (Token<?> token : job1Token) {
-              if (token.getKind().equals(TOKEN_KIND)) {
-                oldCreds.addToken(tokenService, token);
+              if (token.getKind().equals(KMSDelegationToken.TOKEN_KIND)) {
+                oldCreds
+                    .addToken(SecurityUtil.buildTokenService(kmsAddr), token);
               }
             }
             UserGroupInformation.getCurrentUser().addCredentials(oldCreds);
@@ -2450,11 +2326,12 @@ public class TestKMS {
             }
 
             // Using the new DT should succeed.
-            assertEquals(1, newCreds.getAllTokens().size());
-            assertEquals(TOKEN_KIND,
-                newCreds.getToken(tokenService).getKind());
+            Assert.assertEquals(1, newCreds.getAllTokens().size());
+            Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
+                newCreds.getToken(SecurityUtil.buildTokenService(kmsAddr)).
+                    getKind());
             UserGroupInformation.getCurrentUser().addCredentials(newCreds);
-            LOG.info("Credentials now are: {}", UserGroupInformation
+            LOG.info("Credetials now are: {}", UserGroupInformation
                 .getCurrentUser().getCredentials().getAllTokens());
             kp.getKeys();
             return null;
@@ -2480,13 +2357,7 @@ public class TestKMS {
     doKMSWithZK(true, true);
   }
 
-  private <T> T runServerWithZooKeeper(boolean zkDTSM, boolean zkSigner,
-      KMSCallable<T> callable) throws Exception {
-    return runServerWithZooKeeper(zkDTSM, zkSigner, callable, 1);
-  }
-
-  private <T> T runServerWithZooKeeper(boolean zkDTSM, boolean zkSigner,
-      KMSCallable<T> callable, int kmsSize) throws Exception {
+  public void doKMSWithZK(boolean zkDTSM, boolean zkSigner) throws Exception {
     TestingServer zkServer = null;
     try {
       zkServer = new TestingServer();
@@ -2532,265 +2403,43 @@ public class TestKMS {
 
       writeConf(testDir, conf);
 
-      int[] ports = new int[kmsSize];
-      for (int i = 0; i < ports.length; i++) {
-        ports[i] = -1;
-      }
-      return runServer(ports, null, null, testDir, callable);
+      KMSCallable<KeyProvider> c =
+          new KMSCallable<KeyProvider>() {
+        @Override
+        public KeyProvider call() throws Exception {
+          final Configuration conf = new Configuration();
+          conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
+          final URI uri = createKMSUri(getKMSUrl());
+
+          final KeyProvider kp =
+              doAs("SET_KEY_MATERIAL",
+                  new PrivilegedExceptionAction<KeyProvider>() {
+                    @Override
+                    public KeyProvider run() throws Exception {
+                      KeyProvider kp = createProvider(uri, conf);
+                          kp.createKey("k1", new byte[16],
+                              new KeyProvider.Options(conf));
+                          kp.createKey("k2", new byte[16],
+                              new KeyProvider.Options(conf));
+                          kp.createKey("k3", new byte[16],
+                              new KeyProvider.Options(conf));
+                      return kp;
+                    }
+                  });
+          return kp;
+        }
+      };
+
+      runServer(null, null, testDir, c);
     } finally {
       if (zkServer != null) {
         zkServer.stop();
         zkServer.close();
       }
     }
-  }
-
-  public void doKMSWithZK(boolean zkDTSM, boolean zkSigner) throws Exception {
-    KMSCallable<KeyProvider> c =
-        new KMSCallable<KeyProvider>() {
-          @Override
-          public KeyProvider call() throws Exception {
-            final Configuration conf = new Configuration();
-            conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
-            final URI uri = createKMSUri(getKMSUrl());
-
-            final KeyProvider kp =
-                doAs("SET_KEY_MATERIAL",
-                    new PrivilegedExceptionAction<KeyProvider>() {
-                      @Override
-                      public KeyProvider run() throws Exception {
-                        KeyProvider kp = createProvider(uri, conf);
-                        kp.createKey("k1", new byte[16],
-                            new KeyProvider.Options(conf));
-                        kp.createKey("k2", new byte[16],
-                            new KeyProvider.Options(conf));
-                        kp.createKey("k3", new byte[16],
-                            new KeyProvider.Options(conf));
-                        return kp;
-                      }
-                    });
-            return kp;
-          }
-        };
-
-    runServerWithZooKeeper(zkDTSM, zkSigner, c);
-  }
-
-  @Test
-  public void doKMSHAZKWithDelegationTokenAccess() throws Exception {
-    KMSCallable<Void> c = new KMSCallable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        final Configuration conf = new Configuration();
-        conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
-        final URI[] uris = createKMSHAUri(getKMSHAUrl());
-        final Credentials credentials = new Credentials();
-        final String lbUri = generateLoadBalancingKeyProviderUriString();
-        final LoadBalancingKMSClientProvider lbkp =
-            createHAProvider(uris, conf, lbUri);
-        conf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
-        // Login as a Kerberos user principal using keytab.
-        // Connect to KMS to create a delegation token and add it to credentials
-        final String keyName = "k0";
-        doAs("SET_KEY_MATERIAL",
-            new PrivilegedExceptionAction<Void>() {
-              @Override
-              public Void run() throws Exception {
-                KeyProviderDelegationTokenExtension kpdte =
-                    KeyProviderDelegationTokenExtension.
-                        createKeyProviderDelegationTokenExtension(lbkp);
-                kpdte.addDelegationTokens("SET_KEY_MATERIAL", credentials);
-                kpdte.createKey(keyName, new KeyProvider.Options(conf));
-                return null;
-              }
-            });
-
-        assertTokenIdentifierEquals(credentials);
-
-        final LoadBalancingKMSClientProvider lbkp1 =
-            createHAProvider(uris, conf, lbUri);
-        // verify both tokens can be used to authenticate
-        for (Token t : credentials.getAllTokens()) {
-          assertTokenAccess(lbkp1, keyName, t);
-        }
-        return null;
-      }
-    };
-    runServerWithZooKeeper(true, true, c, 2);
-  }
-
-  /**
-   * Assert that the passed in credentials have 2 tokens, of kind
-   * {@link KMSDelegationToken#TOKEN_KIND} and
-   * {@link KMSDelegationToken#TOKEN_LEGACY_KIND}. Assert that the 2 tokens have
-   * the same identifier.
-   */
-  private void assertTokenIdentifierEquals(Credentials credentials)
-      throws IOException {
-    // verify the 2 tokens have the same identifier
-    assertEquals(2, credentials.getAllTokens().size());
-    Token token = null;
-    Token legacyToken = null;
-    for (Token t : credentials.getAllTokens()) {
-      if (KMSDelegationToken.TOKEN_KIND.equals(t.getKind())) {
-        token = t;
-      } else if (KMSDelegationToken.TOKEN_LEGACY_KIND.equals(t.getKind())) {
-        legacyToken = t;
-      }
-    }
-    assertNotNull(token);
-    assertNotNull(legacyToken);
-    final DelegationTokenIdentifier tokenId =
-        (DelegationTokenIdentifier) token.decodeIdentifier();
-    final DelegationTokenIdentifier legacyTokenId =
-        (DelegationTokenIdentifier) legacyToken.decodeIdentifier();
-    assertEquals("KMS DT and legacy dt should have identical identifier",
-        tokenId, legacyTokenId);
-  }
 
-  /**
-   * Tests token access with each providers in the
-   * {@link LoadBalancingKMSClientProvider}. This is to make sure the 2 token
-   * kinds are compatible and can both be used to authenticate.
-   */
-  private void assertTokenAccess(final LoadBalancingKMSClientProvider lbkp,
-      final String keyName, final Token token) throws Exception {
-    UserGroupInformation tokenUgi =
-        UserGroupInformation.createUserForTesting("test", new String[] {});
-    // Verify the tokens can authenticate to any KMS
-    tokenUgi.addToken(token);
-    tokenUgi.doAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws Exception {
-        // Create a kms client with one provider at a time. Must use one
-        // provider so that if it fails to authenticate, it does not fall
-        // back to the next KMS instance.
-        // It should succeed because its delegation token can access any
-        // KMS instances.
-        for (KMSClientProvider provider : lbkp.getProviders()) {
-          if (token.getKind().equals(TOKEN_LEGACY_KIND) && !token.getService()
-              .equals(provider.getDelegationTokenService())) {
-            // Historically known issue: Legacy token can only work with the
-            // key provider specified in the token's Service
-            continue;
-          }
-          LOG.info("Rolling key {} via provider {} with token {}.", keyName,
-              provider, token);
-          provider.rollNewVersion(keyName);
-        }
-        return null;
-      }
-    });
   }
 
-  @Test
-  public void testKMSHAZKDelegationTokenRenewCancel() throws Exception {
-    testKMSHAZKDelegationTokenRenewCancel(TOKEN_KIND);
-  }
-
-  @Test
-  public void testKMSHAZKDelegationTokenRenewCancelLegacy() throws Exception {
-    testKMSHAZKDelegationTokenRenewCancel(TOKEN_LEGACY_KIND);
-  }
-
-  private void testKMSHAZKDelegationTokenRenewCancel(final Text tokenKind)
-      throws Exception {
-    GenericTestUtils.setLogLevel(KMSTokenRenewer.LOG, Level.TRACE);
-    assertTrue(tokenKind == TOKEN_KIND || tokenKind == TOKEN_LEGACY_KIND);
-    KMSCallable<Void> c = new KMSCallable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        final Configuration conf = new Configuration();
-        final URI[] uris = createKMSHAUri(getKMSHAUrl());
-        final Credentials credentials = new Credentials();
-        // Create a UGI without Kerberos auth. It will be authenticated with
-        // delegation token.
-        final UserGroupInformation nonKerberosUgi =
-            UserGroupInformation.getCurrentUser();
-        final String lbUri = generateLoadBalancingKeyProviderUriString();
-        final LoadBalancingKMSClientProvider lbkp =
-            createHAProvider(uris, conf, lbUri);
-        conf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
-        // Login as a Kerberos user principal using keytab.
-        // Connect to KMS to create a delegation token and add it to credentials
-        doAs("SET_KEY_MATERIAL",
-            new PrivilegedExceptionAction<Void>() {
-              @Override
-              public Void run() throws Exception {
-                KeyProviderDelegationTokenExtension kpdte =
-                    KeyProviderDelegationTokenExtension.
-                        createKeyProviderDelegationTokenExtension(lbkp);
-                kpdte.addDelegationTokens("SET_KEY_MATERIAL", credentials);
-                return null;
-              }
-            });
-
-        // Test token renewal and cancellation
-        final Collection<Token<? extends TokenIdentifier>> tokens =
-            credentials.getAllTokens();
-        doAs("SET_KEY_MATERIAL", new PrivilegedExceptionAction<Void>() {
-          @Override
-          public Void run() throws Exception {
-            Assert.assertEquals(2, tokens.size());
-            boolean tokenFound = false;
-            for (Token token : tokens) {
-              if (!tokenKind.equals(token.getKind())) {
-                continue;
-              } else {
-                tokenFound = true;
-              }
-              KMSUtilFaultInjector.set(testInjector);
-              setupConfForToken(token.getKind(), conf, lbUri);
-
-              LOG.info("Testing token: {}", token);
-              long tokenLife = token.renew(conf);
-              LOG.info("Renewed token {}, new lifetime:{}", token, tokenLife);
-              Thread.sleep(10);
-              long newTokenLife = token.renew(conf);
-              LOG.info("Renewed token {}, new lifetime:{}", token,
-                  newTokenLife);
-              assertTrue(newTokenLife > tokenLife);
-
-              boolean canceled = false;
-              // test delegation token cancellation
-              if (!canceled) {
-                token.cancel(conf);
-                LOG.info("Cancelled token {}", token);
-                canceled = true;
-              }
-              assertTrue("token should have been canceled", canceled);
-              try {
-                token.renew(conf);
-                fail("should not be able to renew a canceled token " + token);
-              } catch (Exception e) {
-                LOG.info("Expected exception when renewing token", e);
-              }
-            }
-            assertTrue("Should have found token kind " + tokenKind + " from "
-                + tokens, tokenFound);
-            return null;
-          }
-        });
-        return null;
-      }
-    };
-    runServerWithZooKeeper(true, true, c, 2);
-  }
-
-  /**
-   * Set or unset the key provider configuration based on token kind.
-   */
-  private void setupConfForToken(Text tokenKind, Configuration conf,
-      String lbUri) {
-    if (tokenKind.equals(TOKEN_KIND)) {
-      conf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
-    } else {
-      // conf is only required for legacy tokens to create provider,
-      // new tokens create provider by parsing its own Service field
-      assertEquals(TOKEN_LEGACY_KIND, tokenKind);
-      conf.set(HADOOP_SECURITY_KEY_PROVIDER_PATH, lbUri);
-    }
-  }
 
   @Test
   public void testProxyUserKerb() throws Exception {
@@ -2909,16 +2558,6 @@ public class TestKMS {
 
   @Test
   public void testTGTRenewal() throws Exception {
-    shutdownMiniKdc();
-    try {
-      testTgtRenewalInt();
-    } finally {
-      shutdownMiniKdc();
-      setUpMiniKdc();
-    }
-  }
-
-  private void testTgtRenewalInt() throws Exception {
     Properties kdcConf = MiniKdc.createConf();
     kdcConf.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "3");
     kdcConf.setProperty(MiniKdc.MIN_TICKET_LIFETIME, "3");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/35] hadoop git commit: YARN-8151. Yarn RM Epoch should wrap around. Contributed by Young Chen.

Posted by xy...@apache.org.
YARN-8151. Yarn RM Epoch should wrap around. Contributed by Young Chen.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e6a80e47
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e6a80e47
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e6a80e47

Branch: refs/heads/HDDS-4
Commit: e6a80e476d4348a4373e6dd5792d70edff16516f
Parents: 87c23ef
Author: Inigo Goiri <in...@apache.org>
Authored: Wed May 2 17:23:17 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed May 2 17:23:17 2018 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java       |  4 ++++
 .../src/main/resources/yarn-default.xml           |  7 +++++++
 .../recovery/FileSystemRMStateStore.java          |  4 ++--
 .../recovery/LeveldbRMStateStore.java             |  2 +-
 .../recovery/MemoryRMStateStore.java              |  2 +-
 .../resourcemanager/recovery/RMStateStore.java    | 18 +++++++++++++++++-
 .../resourcemanager/recovery/ZKRMStateStore.java  |  4 ++--
 .../recovery/RMStateStoreTestBase.java            | 14 ++++++++++++++
 .../recovery/TestFSRMStateStore.java              |  1 +
 .../recovery/TestLeveldbRMStateStore.java         |  1 +
 .../recovery/TestZKRMStateStore.java              |  1 +
 11 files changed, 51 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 8aa136d..5ba2e05 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -188,6 +188,10 @@ public class YarnConfiguration extends Configuration {
   public static final String RM_EPOCH = RM_PREFIX + "epoch";
   public static final long DEFAULT_RM_EPOCH = 0L;
 
+  /** The epoch range before wrap around. 0 disables wrap around*/
+  public static final String RM_EPOCH_RANGE = RM_EPOCH + ".range";
+  public static final long DEFAULT_RM_EPOCH_RANGE = 0;
+
   /** The address of the applications manager interface in the RM.*/
   public static final String RM_ADDRESS = 
     RM_PREFIX + "address";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 85915c2..4eb509f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -677,6 +677,13 @@
   </property>
 
   <property>
+    <description>The range of values above base epoch that the RM will use before
+      wrapping around</description>
+    <name>yarn.resourcemanager.epoch.range</name>
+    <value>0</value>
+  </property>
+
+  <property>
     <description>The list of RM nodes in the cluster when HA is
       enabled. See description of yarn.resourcemanager.ha
       .enabled for full details on how this is used.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
index 19297bc..b797283 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
@@ -205,12 +205,12 @@ public class FileSystemRMStateStore extends RMStateStore {
       Epoch epoch = new EpochPBImpl(EpochProto.parseFrom(data));
       currentEpoch = epoch.getEpoch();
       // increment epoch and store it
-      byte[] storeData = Epoch.newInstance(currentEpoch + 1).getProto()
+      byte[] storeData = Epoch.newInstance(nextEpoch(currentEpoch)).getProto()
           .toByteArray();
       updateFile(epochNodePath, storeData, false);
     } else {
       // initialize epoch file with 1 for the next time.
-      byte[] storeData = Epoch.newInstance(currentEpoch + 1).getProto()
+      byte[] storeData = Epoch.newInstance(nextEpoch(currentEpoch)).getProto()
           .toByteArray();
       writeFileWithRetries(epochNodePath, storeData, false);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
index 36a8dfa..e7fb02f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
@@ -259,7 +259,7 @@ public class LeveldbRMStateStore extends RMStateStore {
       if (data != null) {
         currentEpoch = EpochProto.parseFrom(data).getEpoch();
       }
-      EpochProto proto = Epoch.newInstance(currentEpoch + 1).getProto();
+      EpochProto proto = Epoch.newInstance(nextEpoch(currentEpoch)).getProto();
       db.put(dbKeyBytes, proto.toByteArray());
     } catch (DBException e) {
       throw new IOException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
index 5041000..219e10a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
@@ -59,7 +59,7 @@ public class MemoryRMStateStore extends RMStateStore {
   @Override
   public synchronized long getAndIncrementEpoch() throws Exception {
     long currentEpoch = epoch;
-    epoch = epoch + 1;
+    epoch = nextEpoch(epoch);
     return currentEpoch;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
index b4dd378..242b5d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
@@ -104,6 +104,7 @@ public abstract class RMStateStore extends AbstractService {
   protected static final String VERSION_NODE = "RMVersionNode";
   protected static final String EPOCH_NODE = "EpochNode";
   protected long baseEpoch;
+  private long epochRange;
   protected ResourceManager resourceManager;
   private final ReadLock readLock;
   private final WriteLock writeLock;
@@ -732,6 +733,8 @@ public abstract class RMStateStore extends AbstractService {
     // read the base epoch value from conf
     baseEpoch = conf.getLong(YarnConfiguration.RM_EPOCH,
         YarnConfiguration.DEFAULT_RM_EPOCH);
+    epochRange = conf.getLong(YarnConfiguration.RM_EPOCH_RANGE,
+        YarnConfiguration.DEFAULT_RM_EPOCH_RANGE);
     initInternal(conf);
   }
 
@@ -818,7 +821,20 @@ public abstract class RMStateStore extends AbstractService {
    * Get the current epoch of RM and increment the value.
    */
   public abstract long getAndIncrementEpoch() throws Exception;
-  
+
+  /**
+   * Compute the next epoch value by incrementing by one.
+   * Wraps around if the epoch range is exceeded so that
+   * when federation is enabled epoch collisions can be avoided.
+   */
+  protected long nextEpoch(long epoch){
+    long epochVal = epoch - baseEpoch + 1;
+    if (epochRange > 0) {
+      epochVal %= epochRange;
+    }
+    return  epochVal + baseEpoch;
+  }
+
   /**
    * Blocking API
    * The derived class must recover state from the store and return a new 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index 9073910..de1f1ad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -491,13 +491,13 @@ public class ZKRMStateStore extends RMStateStore {
       Epoch epoch = new EpochPBImpl(EpochProto.parseFrom(data));
       currentEpoch = epoch.getEpoch();
       // increment epoch and store it
-      byte[] storeData = Epoch.newInstance(currentEpoch + 1).getProto()
+      byte[] storeData = Epoch.newInstance(nextEpoch(currentEpoch)).getProto()
           .toByteArray();
       zkManager.safeSetData(epochNodePath, storeData, -1, zkAcl,
           fencingNodePath);
     } else {
       // initialize epoch node with 1 for the next time.
-      byte[] storeData = Epoch.newInstance(currentEpoch + 1).getProto()
+      byte[] storeData = Epoch.newInstance(nextEpoch(currentEpoch)).getProto()
           .toByteArray();
       zkManager.safeCreate(epochNodePath, storeData, zkAcl,
           CreateMode.PERSISTENT, zkAcl, fencingNodePath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
index 957d4ce..3454d72 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
@@ -94,6 +94,8 @@ public class RMStateStoreTestBase {
 
   protected final long epoch = 10L;
 
+  private final long epochRange = 10L;
+
   static class TestDispatcher implements Dispatcher, EventHandler<Event> {
 
     ApplicationAttemptId attemptId;
@@ -141,6 +143,10 @@ public class RMStateStoreTestBase {
     boolean attemptExists(RMAppAttempt attempt) throws Exception;
   }
 
+  public long getEpochRange() {
+    return epochRange;
+  }
+
   void waitNotify(TestDispatcher dispatcher) {
     long startTime = System.currentTimeMillis();
     while(!dispatcher.notified) {
@@ -576,6 +582,14 @@ public class RMStateStoreTestBase {
     
     long thirdTimeEpoch = store.getAndIncrementEpoch();
     Assert.assertEquals(epoch + 2, thirdTimeEpoch);
+
+    for (int i = 0; i < epochRange; ++i) {
+      store.getAndIncrementEpoch();
+    }
+    long wrappedEpoch = store.getAndIncrementEpoch();
+    // Epoch should have wrapped around and then incremented once for a total
+    // of + 3
+    Assert.assertEquals(epoch + 3, wrappedEpoch);
   }
 
   public void testAppDeletion(RMStateStoreHelper stateStoreHelper)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
index fe4a701..14f5404 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
@@ -118,6 +118,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       conf.setLong(YarnConfiguration.FS_RM_STATE_STORE_RETRY_INTERVAL_MS,
               900L);
       conf.setLong(YarnConfiguration.RM_EPOCH, epoch);
+      conf.setLong(YarnConfiguration.RM_EPOCH_RANGE, getEpochRange());
       if (adminCheckEnable) {
         conf.setBoolean(
           YarnConfiguration.YARN_INTERMEDIATE_DATA_ENCRYPTION, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
index afd0c77..576ee7f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
@@ -83,6 +83,7 @@ public class TestLeveldbRMStateStore extends RMStateStoreTestBase {
   @Test(timeout = 60000)
   public void testEpoch() throws Exception {
     conf.setLong(YarnConfiguration.RM_EPOCH, epoch);
+    conf.setLong(YarnConfiguration.RM_EPOCH_RANGE, getEpochRange());
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testEpoch(tester);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a80e47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
index d8718e0..4cba266 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
@@ -210,6 +210,7 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
           curatorTestingServer.getConnectString());
       conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
       conf.setLong(YarnConfiguration.RM_EPOCH, epoch);
+      conf.setLong(YarnConfiguration.RM_EPOCH_RANGE, getEpochRange());
       this.store = new TestZKRMStateStoreInternal(conf, workingZnode);
       return this.store;
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/35] hadoop git commit: YARN-8226. Improved anti-affinity description in YARN Service doc. Contributed by Gour Saha

Posted by xy...@apache.org.
YARN-8226. Improved anti-affinity description in YARN Service doc.
           Contributed by Gour Saha


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/76987372
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/76987372
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/76987372

Branch: refs/heads/HDDS-4
Commit: 7698737207b01e80b1be2b4df60363f952a1c2b4
Parents: 7fe3214
Author: Eric Yang <ey...@apache.org>
Authored: Thu May 3 13:35:40 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu May 3 13:35:40 2018 -0400

----------------------------------------------------------------------
 .../main/resources/definition/YARN-Services-Examples.md   | 10 +++++++---
 .../src/site/markdown/yarn-service/YarnServiceAPI.md      | 10 +++++++---
 2 files changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76987372/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md
index a4ef2d2..83e558c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md
@@ -351,9 +351,13 @@ POST URL - http://localhost:8088/app/v1/services
 ##### GET Response JSON
 GET URL - http://localhost:8088/app/v1/services/hello-world
 
-Note, that the 3 containers will come up on 3 different nodes. If there are less
-than 3 NMs running in the cluster, then all 3 container requests will not be
-fulfilled and the service will be in non-STABLE state.
+Note, for an anti-affinity component no more than 1 container will be allocated
+in a specific node. In this example, 3 containers have been requested by
+component "hello". All 3 containers were allocated because the cluster had 3 or
+more NMs. If the cluster had less than 3 NMs then less than 3 containers would
+be allocated. In cases when the number of allocated containers are less than the
+number of requested containers, the component and the service will be in
+non-STABLE state.
 
 ```json
 {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76987372/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
index 496c1a1..fab33c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
@@ -766,9 +766,13 @@ POST URL - http://localhost:8088/app/v1/services
 ##### GET Response JSON
 GET URL - http://localhost:8088/app/v1/services/hello-world
 
-Note, that the 3 containers will come up on 3 different nodes. If there are less
-than 3 NMs running in the cluster, then all 3 container requests will not be
-fulfilled and the service will be in non-STABLE state.
+Note, for an anti-affinity component no more than 1 container will be allocated
+in a specific node. In this example, 3 containers have been requested by
+component "hello". All 3 containers were allocated because the cluster had 3 or
+more NMs. If the cluster had less than 3 NMs then less than 3 containers would
+be allocated. In cases when the number of allocated containers are less than the
+number of requested containers, the component and the service will be in
+non-STABLE state.
 
 ```json
 {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/35] hadoop git commit: Revert "HDFS-13430. Fix TestEncryptionZonesWithKMS failure due to HADOOP-14445."

Posted by xy...@apache.org.
Revert "HDFS-13430. Fix TestEncryptionZonesWithKMS failure due to HADOOP-14445."

This reverts commit 650359371175fba416331e73aa03d2a96ccb90e5.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/118bd758
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/118bd758
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/118bd758

Branch: refs/heads/HDDS-4
Commit: 118bd7580583e31bf643b642a2fbc9556177b906
Parents: 5b11b9f
Author: Xiao Chen <xi...@apache.org>
Authored: Fri May 4 20:19:34 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Mon May 7 13:32:27 2018 -0700

----------------------------------------------------------------------
 .../src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/118bd758/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index 51c6c4e..6f9ef29 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -110,7 +110,6 @@ import org.junit.Test;
 import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.KMS_CLIENT_COPY_LEGACY_TOKEN_KEY;
 import static org.junit.Assert.assertNotNull;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyLong;
@@ -193,8 +192,6 @@ public class TestEncryptionZones {
     // Lower the batch size for testing
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
         2);
-    // disable kms client copy legacy token logic because it's irrelevant.
-    conf.setBoolean(KMS_CLIENT_COPY_LEGACY_TOKEN_KEY, false);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     cluster.waitActive();
     Logger.getLogger(EncryptionZoneManager.class).setLevel(Level.TRACE);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/35] hadoop git commit: YARN-6385. Fix checkstyle warnings in TestFileSystemApplicationHistoryStore

Posted by xy...@apache.org.
YARN-6385. Fix checkstyle warnings in TestFileSystemApplicationHistoryStore

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3265b551
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3265b551
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3265b551

Branch: refs/heads/HDDS-4
Commit: 3265b55119d39ecbda6d75be04a9a1bf59c631f1
Parents: e07156e
Author: Yiqun Lin <yi...@vipshop.com>
Authored: Wed May 2 18:14:02 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed May 2 18:14:02 2018 +0900

----------------------------------------------------------------------
 .../TestFileSystemApplicationHistoryStore.java    | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3265b551/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
index df4adbe..6b068c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
@@ -272,20 +272,20 @@ public class TestFileSystemApplicationHistoryStore extends
     tearDown();
 
     // Setup file system to inject startup conditions
-    FileSystem fs = spy(new RawLocalFileSystem());
+    FileSystem fileSystem = spy(new RawLocalFileSystem());
     FileStatus fileStatus = Mockito.mock(FileStatus.class);
     doReturn(true).when(fileStatus).isDirectory();
-    doReturn(fileStatus).when(fs).getFileStatus(any(Path.class));
+    doReturn(fileStatus).when(fileSystem).getFileStatus(any(Path.class));
 
     try {
-      initAndStartStore(fs);
+      initAndStartStore(fileSystem);
     } catch (Exception e) {
       Assert.fail("Exception should not be thrown: " + e);
     }
 
     // Make sure that directory creation was not attempted
     verify(fileStatus, never()).isDirectory();
-    verify(fs, times(1)).mkdirs(any(Path.class));
+    verify(fileSystem, times(1)).mkdirs(any(Path.class));
   }
 
   @Test
@@ -294,14 +294,14 @@ public class TestFileSystemApplicationHistoryStore extends
     tearDown();
 
     // Setup file system to inject startup conditions
-    FileSystem fs = spy(new RawLocalFileSystem());
+    FileSystem fileSystem = spy(new RawLocalFileSystem());
     FileStatus fileStatus = Mockito.mock(FileStatus.class);
     doReturn(false).when(fileStatus).isDirectory();
-    doReturn(fileStatus).when(fs).getFileStatus(any(Path.class));
-    doThrow(new IOException()).when(fs).mkdirs(any(Path.class));
+    doReturn(fileStatus).when(fileSystem).getFileStatus(any(Path.class));
+    doThrow(new IOException()).when(fileSystem).mkdirs(any(Path.class));
 
     try {
-      initAndStartStore(fs);
+      initAndStartStore(fileSystem);
       Assert.fail("Exception should have been thrown");
     } catch (Exception e) {
       // Expected failure
@@ -309,6 +309,6 @@ public class TestFileSystemApplicationHistoryStore extends
 
     // Make sure that directory creation was attempted
     verify(fileStatus, never()).isDirectory();
-    verify(fs, times(1)).mkdirs(any(Path.class));
+    verify(fileSystem, times(1)).mkdirs(any(Path.class));
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/35] hadoop git commit: HDFS-13488. RBF: Reject requests when a Router is overloaded. Contributed by Inigo Goiri.

Posted by xy...@apache.org.
HDFS-13488. RBF: Reject requests when a Router is overloaded. Contributed by Inigo Goiri.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/37269261
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/37269261
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/37269261

Branch: refs/heads/HDDS-4
Commit: 37269261d1232bc71708f30c76193188258ef4bd
Parents: 8f42daf
Author: Yiqun Lin <yq...@apache.org>
Authored: Wed May 2 14:49:39 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Wed May 2 14:49:39 2018 +0800

----------------------------------------------------------------------
 .../federation/metrics/FederationRPCMBean.java  |   2 +
 .../metrics/FederationRPCMetrics.java           |  10 +
 .../FederationRPCPerformanceMonitor.java        |   5 +
 .../server/federation/router/RBFConfigKeys.java |   3 +
 .../federation/router/RouterRpcClient.java      |  31 ++-
 .../federation/router/RouterRpcMonitor.java     |   6 +
 .../federation/router/RouterRpcServer.java      |  11 +-
 .../router/RouterSafeModeException.java         |  53 ----
 .../src/main/resources/hdfs-rbf-default.xml     |   9 +
 .../server/federation/FederationTestUtils.java  |   2 +-
 .../server/federation/StateStoreDFSCluster.java |  28 +++
 .../router/TestRouterClientRejectOverload.java  | 243 +++++++++++++++++++
 .../router/TestRouterRPCClientRetries.java      |  51 +---
 .../federation/router/TestRouterSafemode.java   |   3 +-
 14 files changed, 349 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java
index 3e031fe..973c398 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java
@@ -40,6 +40,8 @@ public interface FederationRPCMBean {
 
   long getProxyOpFailureStandby();
 
+  long getProxyOpFailureClientOverloaded();
+
   long getProxyOpNotImplemented();
 
   long getProxyOpRetries();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
index 94d3383..9ab4e5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
@@ -54,6 +54,8 @@ public class FederationRPCMetrics implements FederationRPCMBean {
   private MutableCounterLong proxyOpFailureStandby;
   @Metric("Number of operations to hit a standby NN")
   private MutableCounterLong proxyOpFailureCommunicate;
+  @Metric("Number of operations to hit a client overloaded Router")
+  private MutableCounterLong proxyOpFailureClientOverloaded;
   @Metric("Number of operations not implemented")
   private MutableCounterLong proxyOpNotImplemented;
   @Metric("Number of operation retries")
@@ -118,6 +120,14 @@ public class FederationRPCMetrics implements FederationRPCMBean {
     return proxyOpFailureCommunicate.value();
   }
 
+  public void incrProxyOpFailureClientOverloaded() {
+    proxyOpFailureClientOverloaded.incr();
+  }
+
+  @Override
+  public long getProxyOpFailureClientOverloaded() {
+    return proxyOpFailureClientOverloaded.value();
+  }
 
   public void incrProxyOpNotImplemented() {
     proxyOpNotImplemented.incr();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
index 547ebb5..2c2741e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
@@ -154,6 +154,11 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
   }
 
   @Override
+  public void proxyOpFailureClientOverloaded() {
+    metrics.incrProxyOpFailureClientOverloaded();
+  }
+
+  @Override
   public void proxyOpNotImplemented() {
     metrics.incrProxyOpNotImplemented();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
index 170b876..363db20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
@@ -113,6 +113,9 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
   public static final String DFS_ROUTER_CLIENT_MAX_ATTEMPTS =
       FEDERATION_ROUTER_PREFIX + "client.retry.max.attempts";
   public static final int DFS_ROUTER_CLIENT_MAX_ATTEMPTS_DEFAULT = 3;
+  public static final String DFS_ROUTER_CLIENT_REJECT_OVERLOAD =
+      FEDERATION_ROUTER_PREFIX + "client.reject.overload";
+  public static final boolean DFS_ROUTER_CLIENT_REJECT_OVERLOAD_DEFAULT = false;
 
   // HDFS Router State Store connection
   public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
index 513e867..3eb7241 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
@@ -35,13 +35,16 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -98,7 +101,7 @@ public class RouterRpcClient {
   /** Connection pool to the Namenodes per user for performance. */
   private final ConnectionManager connectionManager;
   /** Service to run asynchronous calls. */
-  private final ExecutorService executorService;
+  private final ThreadPoolExecutor executorService;
   /** Retry policy for router -> NN communication. */
   private final RetryPolicy retryPolicy;
   /** Optional perf monitor. */
@@ -131,8 +134,16 @@ public class RouterRpcClient {
     ThreadFactory threadFactory = new ThreadFactoryBuilder()
         .setNameFormat("RPC Router Client-%d")
         .build();
-    this.executorService = Executors.newFixedThreadPool(
-        numThreads, threadFactory);
+    BlockingQueue<Runnable> workQueue;
+    if (conf.getBoolean(
+        RBFConfigKeys.DFS_ROUTER_CLIENT_REJECT_OVERLOAD,
+        RBFConfigKeys.DFS_ROUTER_CLIENT_REJECT_OVERLOAD_DEFAULT)) {
+      workQueue = new ArrayBlockingQueue<>(numThreads);
+    } else {
+      workQueue = new LinkedBlockingQueue<>();
+    }
+    this.executorService = new ThreadPoolExecutor(numThreads, numThreads,
+        0L, TimeUnit.MILLISECONDS, workQueue, threadFactory);
 
     this.rpcMonitor = monitor;
 
@@ -1106,6 +1117,16 @@ public class RouterRpcClient {
       }
 
       return results;
+    } catch (RejectedExecutionException e) {
+      if (rpcMonitor != null) {
+        rpcMonitor.proxyOpFailureClientOverloaded();
+      }
+      int active = executorService.getActiveCount();
+      int total = executorService.getMaximumPoolSize();
+      String msg = "Not enough client threads " + active + "/" + total;
+      LOG.error(msg);
+      throw new StandbyException(
+          "Router " + routerId + " is overloaded: " + msg);
     } catch (InterruptedException ex) {
       LOG.error("Unexpected error while invoking API: {}", ex.getMessage());
       throw new IOException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java
index df9aa11..7af71af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java
@@ -76,6 +76,12 @@ public interface RouterRpcMonitor {
   void proxyOpFailureCommunicate();
 
   /**
+   * Failed to proxy an operation to a Namenode because the client was
+   * overloaded.
+   */
+  void proxyOpFailureClientOverloaded();
+
+  /**
    * Failed to proxy an operation because it is not implemented.
    */
   void proxyOpNotImplemented();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 21f26d0..6b466b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -289,7 +289,6 @@ public class RouterRpcServer extends AbstractService
     // We don't want the server to log the full stack trace for some exceptions
     this.rpcServer.addTerseExceptions(
         RemoteException.class,
-        StandbyException.class,
         SafeModeException.class,
         FileNotFoundException.class,
         FileAlreadyExistsException.class,
@@ -298,6 +297,9 @@ public class RouterRpcServer extends AbstractService
         NotReplicatedYetException.class,
         IOException.class);
 
+    this.rpcServer.addSuppressedLoggingExceptions(
+        StandbyException.class);
+
     // The RPC-server port can be ephemeral... ensure we have the correct info
     InetSocketAddress listenAddress = this.rpcServer.getListenerAddress();
     this.rpcAddress = new InetSocketAddress(
@@ -413,7 +415,7 @@ public class RouterRpcServer extends AbstractService
    * @throws UnsupportedOperationException If the operation is not supported.
    */
   protected void checkOperation(OperationCategory op, boolean supported)
-      throws RouterSafeModeException, UnsupportedOperationException {
+      throws StandbyException, UnsupportedOperationException {
     checkOperation(op);
 
     if (!supported) {
@@ -435,7 +437,7 @@ public class RouterRpcServer extends AbstractService
    *                           client requests.
    */
   protected void checkOperation(OperationCategory op)
-      throws RouterSafeModeException {
+      throws StandbyException {
     // Log the function we are currently calling.
     if (rpcMonitor != null) {
       rpcMonitor.startOp();
@@ -459,7 +461,8 @@ public class RouterRpcServer extends AbstractService
       if (rpcMonitor != null) {
         rpcMonitor.routerFailureSafemode();
       }
-      throw new RouterSafeModeException(router.getRouterId(), op);
+      throw new StandbyException("Router " + router.getRouterId() +
+          " is in safe mode and cannot handle " + op + " requests");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafeModeException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafeModeException.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafeModeException.java
deleted file mode 100644
index 7a78b5b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSafeModeException.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.hdfs.server.federation.router;
-
-import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
-import org.apache.hadoop.ipc.StandbyException;
-
-/**
- * Exception that the Router throws when it is in safe mode. This extends
- * {@link StandbyException} for the client to try another Router when it gets
- * this exception.
- */
-public class RouterSafeModeException extends StandbyException {
-
-  private static final long serialVersionUID = 453568188334993493L;
-
-  /** Identifier of the Router that generated this exception. */
-  private final String routerId;
-
-  /**
-   * Build a new Router safe mode exception.
-   * @param router Identifier of the Router.
-   * @param op Category of the operation (READ/WRITE).
-   */
-  public RouterSafeModeException(String router, OperationCategory op) {
-    super("Router " + router + " is in safe mode and cannot handle " + op
-        + " requests.");
-    this.routerId = router;
-  }
-
-  /**
-   * Get the id of the Router that generated this exception.
-   * @return Id of the Router that generated this exception.
-   */
-  public String getRouterId() {
-    return this.routerId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
index 92f899d..8806cb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
@@ -431,4 +431,13 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.client.reject.overload</name>
+    <value>false</value>
+    <description>
+      Set to true to reject client requests when we run out of RPC client
+      threads.
+    </description>
+  </property>
+
 </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
index ed1428a..ce320f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
@@ -59,7 +59,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.Whitebox;
+import org.mockito.internal.util.reflection.Whitebox;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.java
index bf63b18..9d56f13 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/StateStoreDFSCluster.java
@@ -28,6 +28,10 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
@@ -37,6 +41,7 @@ import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Test utility to mimic a federated HDFS cluster with a router and a state
@@ -145,4 +150,27 @@ public class StateStoreDFSCluster extends MiniRouterDFSCluster {
     entries.add(entry);
     return entries;
   }
+
+  /**
+   * Get the client configuration which targets all the Routers. It uses the HA
+   * setup to fails over between them.
+   * @return Configuration for the client which uses two routers.
+   */
+  public Configuration getRouterClientConf() {
+    List<RouterContext> routers = getRouters();
+    Configuration clientConf = DFSTestUtil.newHAConfiguration("fed");
+    int i = 0;
+    List<String> names = new ArrayList<>(routers.size());
+    for (RouterContext routerContext : routers) {
+      String name = "r" + i++;
+      clientConf.set(
+          DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + ".fed." + name,
+          "localhost:" + routerContext.getRpcPort());
+      names.add(name);
+    }
+    clientConf.set(DFSUtil.addKeySuffixes(
+        HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX, "fed"),
+        StringUtils.join(",", names));
+    return clientConf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterClientRejectOverload.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterClientRejectOverload.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterClientRejectOverload.java
new file mode 100644
index 0000000..3c51e13
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterClientRejectOverload.java
@@ -0,0 +1,243 @@
+/**
+ * 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.hdfs.server.federation.router;
+
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.simulateSlowNamenode;
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCMetrics;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.StandbyException;
+import org.junit.After;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Test the Router overload control which rejects requests when the RPC client
+ * is overloaded. This feature is managed by
+ * {@link RBFConfigKeys#DFS_ROUTER_CLIENT_REJECT_OVERLOAD}.
+ */
+public class TestRouterClientRejectOverload {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRouterClientRejectOverload.class);
+
+  private StateStoreDFSCluster cluster;
+
+  @After
+  public void cleanup() {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  private void setupCluster(boolean overloadControl) throws Exception {
+    // Build and start a federated cluster
+    cluster = new StateStoreDFSCluster(false, 2);
+    Configuration routerConf = new RouterConfigBuilder()
+        .stateStore()
+        .metrics()
+        .admin()
+        .rpc()
+        .build();
+
+    // Reduce the number of RPC clients threads to overload the Router easy
+    routerConf.setInt(RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE, 4);
+    // Overload control
+    routerConf.setBoolean(
+        RBFConfigKeys.DFS_ROUTER_CLIENT_REJECT_OVERLOAD, overloadControl);
+
+    // No need for datanodes as we use renewLease() for testing
+    cluster.setNumDatanodesPerNameservice(0);
+
+    cluster.addRouterOverrides(routerConf);
+    cluster.startCluster();
+    cluster.startRouters();
+    cluster.waitClusterUp();
+  }
+
+  @Test
+  public void testWithoutOverloadControl() throws Exception {
+    setupCluster(false);
+
+    // Nobody should get overloaded
+    testOverloaded(0);
+
+    // Set subcluster 0 as slow
+    MiniDFSCluster dfsCluster = cluster.getCluster();
+    NameNode nn0 = dfsCluster.getNameNode(0);
+    simulateSlowNamenode(nn0, 1);
+
+    // Nobody should get overloaded, but it will be really slow
+    testOverloaded(0);
+
+    // No rejected requests expected
+    for (RouterContext router : cluster.getRouters()) {
+      FederationRPCMetrics rpcMetrics =
+          router.getRouter().getRpcServer().getRPCMetrics();
+      assertEquals(0, rpcMetrics.getProxyOpFailureClientOverloaded());
+    }
+  }
+
+  @Test
+  public void testOverloadControl() throws Exception {
+    setupCluster(true);
+
+    List<RouterContext> routers = cluster.getRouters();
+    FederationRPCMetrics rpcMetrics0 =
+        routers.get(0).getRouter().getRpcServer().getRPCMetrics();
+    FederationRPCMetrics rpcMetrics1 =
+        routers.get(1).getRouter().getRpcServer().getRPCMetrics();
+
+    // Nobody should get overloaded
+    testOverloaded(0);
+    assertEquals(0, rpcMetrics0.getProxyOpFailureClientOverloaded());
+    assertEquals(0, rpcMetrics1.getProxyOpFailureClientOverloaded());
+
+    // Set subcluster 0 as slow
+    MiniDFSCluster dfsCluster = cluster.getCluster();
+    NameNode nn0 = dfsCluster.getNameNode(0);
+    simulateSlowNamenode(nn0, 1);
+
+    // The subcluster should be overloaded now and reject 4-5 requests
+    testOverloaded(4, 6);
+    assertTrue(rpcMetrics0.getProxyOpFailureClientOverloaded()
+        + rpcMetrics1.getProxyOpFailureClientOverloaded() >= 4);
+
+    // Client using HA with 2 Routers
+    // A single Router gets overloaded, but 2 will handle it
+    Configuration clientConf = cluster.getRouterClientConf();
+
+    // Each Router should get a similar number of ops (>=8) out of 2*10
+    long iniProxyOps0 = rpcMetrics0.getProxyOps();
+    long iniProxyOps1 = rpcMetrics1.getProxyOps();
+    testOverloaded(0, 0, new URI("hdfs://fed/"), clientConf, 10);
+    long proxyOps0 = rpcMetrics0.getProxyOps() - iniProxyOps0;
+    long proxyOps1 = rpcMetrics1.getProxyOps() - iniProxyOps1;
+    assertEquals(2 * 10, proxyOps0 + proxyOps1);
+    assertTrue(proxyOps0 + " operations: not distributed", proxyOps0 >= 8);
+    assertTrue(proxyOps1 + " operations: not distributed", proxyOps1 >= 8);
+  }
+
+  private void testOverloaded(int expOverload) throws Exception {
+    testOverloaded(expOverload, expOverload);
+  }
+
+  private void testOverloaded(int expOverloadMin, int expOverloadMax)
+      throws Exception {
+    RouterContext routerContext = cluster.getRandomRouter();
+    URI address = routerContext.getFileSystemURI();
+    Configuration conf = new HdfsConfiguration();
+    testOverloaded(expOverloadMin, expOverloadMax, address, conf, 10);
+  }
+
+  /**
+   * Test if the Router gets overloaded by submitting requests in parallel.
+   * We check how many requests got rejected at the end.
+   * @param expOverloadMin Min number of requests expected as overloaded.
+   * @param expOverloadMax Max number of requests expected as overloaded.
+   * @param address Destination address.
+   * @param conf Configuration of the client.
+   * @param numOps Number of operations to submit.
+   * @throws Exception If it cannot perform the test.
+   */
+  private void testOverloaded(int expOverloadMin, int expOverloadMax,
+      final URI address, final Configuration conf, final int numOps)
+          throws Exception {
+
+    // Submit renewLease() ops which go to all subclusters
+    final AtomicInteger overloadException = new AtomicInteger();
+    ExecutorService exec = Executors.newFixedThreadPool(numOps);
+    List<Future<?>> futures = new ArrayList<>();
+    for (int i = 0; i < numOps; i++) {
+      // Stagger the operations a little (50ms)
+      final int sleepTime = i * 50;
+      Future<?> future = exec.submit(new Runnable() {
+        @Override
+        public void run() {
+          DFSClient routerClient = null;
+          try {
+            Thread.sleep(sleepTime);
+            routerClient = new DFSClient(address, conf);
+            String clientName = routerClient.getClientName();
+            ClientProtocol routerProto = routerClient.getNamenode();
+            routerProto.renewLease(clientName);
+          } catch (RemoteException re) {
+            IOException ioe = re.unwrapRemoteException();
+            assertTrue("Wrong exception: " + ioe,
+                ioe instanceof StandbyException);
+            assertExceptionContains("is overloaded", ioe);
+            overloadException.incrementAndGet();
+          } catch (IOException e) {
+            fail("Unexpected exception: " + e);
+          } catch (InterruptedException e) {
+            fail("Cannot sleep: " + e);
+          } finally {
+            if (routerClient != null) {
+              try {
+                routerClient.close();
+              } catch (IOException e) {
+                LOG.error("Cannot close the client");
+              }
+            }
+          }
+        }
+      });
+      futures.add(future);
+    }
+    // Wait until all the requests are done
+    while (!futures.isEmpty()) {
+      futures.remove(0).get();
+    }
+    exec.shutdown();
+
+    int num = overloadException.get();
+    if (expOverloadMin == expOverloadMax) {
+      assertEquals(expOverloadMin, num);
+    } else {
+      assertTrue("Expected >=" + expOverloadMin + " but was " + num,
+          num >= expOverloadMin);
+      assertTrue("Expected <=" + expOverloadMax + " but was " + num,
+          num <= expOverloadMax);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java
index 372dd3b..e5ab3ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java
@@ -17,13 +17,13 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.simulateSlowNamenode;
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.apache.hadoop.test.GenericTestUtils.waitFor;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
 import java.util.List;
@@ -44,13 +44,8 @@ import org.apache.hadoop.hdfs.server.federation.metrics.NamenodeBeanMetrics;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
 import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
-import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.Whitebox;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.After;
@@ -58,10 +53,6 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Supplier;
 
@@ -70,9 +61,6 @@ import com.google.common.base.Supplier;
  */
 public class TestRouterRPCClientRetries {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestRouterRPCClientRetries.class);
-
   private static StateStoreDFSCluster cluster;
   private static NamenodeContext nnContext1;
   private static RouterContext routerContext;
@@ -144,7 +132,7 @@ public class TestRouterRPCClientRetries {
       fail("Should have thrown RemoteException error.");
     } catch (RemoteException e) {
       String ns0 = cluster.getNameservices().get(0);
-      GenericTestUtils.assertExceptionContains(
+      assertExceptionContains(
           "No namenode available under nameservice " + ns0, e);
     }
 
@@ -212,14 +200,14 @@ public class TestRouterRPCClientRetries {
     // Making subcluster0 slow to reply, should only get DNs from nn1
     MiniDFSCluster dfsCluster = cluster.getCluster();
     NameNode nn0 = dfsCluster.getNameNode(0);
-    simulateNNSlow(nn0);
+    simulateSlowNamenode(nn0, 3);
     waitUpdateLiveNodes(jsonString2, metrics);
     final String jsonString3 = metrics.getLiveNodes();
     assertEquals(2, getNumDatanodes(jsonString3));
 
     // Making subcluster1 slow to reply, shouldn't get any DNs
     NameNode nn1 = dfsCluster.getNameNode(1);
-    simulateNNSlow(nn1);
+    simulateSlowNamenode(nn1, 3);
     waitUpdateLiveNodes(jsonString3, metrics);
     final String jsonString4 = metrics.getLiveNodes();
     assertEquals(0, getNumDatanodes(jsonString4));
@@ -249,36 +237,11 @@ public class TestRouterRPCClientRetries {
   private static void waitUpdateLiveNodes(
       final String oldValue, final NamenodeBeanMetrics metrics)
           throws Exception {
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+    waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
         return !oldValue.equals(metrics.getLiveNodes());
       }
     }, 500, 5 * 1000);
   }
-
-  /**
-   * Simulate that a Namenode is slow by adding a sleep to the check operation
-   * in the NN.
-   * @param nn Namenode to simulate slow.
-   * @throws Exception If we cannot add the sleep time.
-   */
-  private static void simulateNNSlow(final NameNode nn) throws Exception {
-    FSNamesystem namesystem = nn.getNamesystem();
-    HAContext haContext = namesystem.getHAContext();
-    HAContext spyHAContext = spy(haContext);
-    doAnswer(new Answer<Object>() {
-      @Override
-      public Object answer(InvocationOnMock invocation) throws Throwable {
-        LOG.info("Simulating slow namenode {}", invocation.getMock());
-        try {
-          Thread.sleep(3 * 1000);
-        } catch(InterruptedException e) {
-          LOG.error("Simulating a slow namenode aborted");
-        }
-        return null;
-      }
-    }).when(spyHAContext).checkOperation(any(OperationCategory.class));
-    Whitebox.setInternalState(namesystem, "haContext", spyHAContext);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37269261/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java
index e5d8348..f16ceb5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java
@@ -33,6 +33,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
@@ -187,7 +188,7 @@ public class TestRouterSafemode {
     try {
       router.getRpcServer().delete("/testfile.txt", true);
       fail("We should have thrown a safe mode exception");
-    } catch (RouterSafeModeException sme) {
+    } catch (StandbyException sme) {
       exception = true;
     }
     assertTrue("We should have thrown a safe mode exception", exception);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/35] hadoop git commit: HADOOP-14841 Kms client should disconnect if unable to get output stream from connection. Contributed by Rushabh S Shah

Posted by xy...@apache.org.
HADOOP-14841 Kms client should disconnect if unable to get output stream from connection. Contributed by Rushabh S Shah


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4cdbdce7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4cdbdce7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4cdbdce7

Branch: refs/heads/HDDS-4
Commit: 4cdbdce752e192b45c2b9756c2d4bd24ceffdabd
Parents: 96c843f
Author: Rushabh Shah <sh...@apache.org>
Authored: Fri May 4 15:36:13 2018 -0500
Committer: Rushabh Shah <sh...@apache.org>
Committed: Fri May 4 15:36:13 2018 -0500

----------------------------------------------------------------------
 .../hadoop/crypto/key/kms/KMSClientProvider.java       | 13 +++++++++++--
 1 file changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4cdbdce7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
index f97fde7..45097ef 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
@@ -447,12 +447,21 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       int expectedResponse, Class<T> klass, int authRetryCount)
       throws IOException {
     T ret = null;
+    OutputStream os = null;
     try {
       if (jsonOutput != null) {
-        writeJson(jsonOutput, conn.getOutputStream());
+        os = conn.getOutputStream();
+        writeJson(jsonOutput, os);
       }
     } catch (IOException ex) {
-      IOUtils.closeStream(conn.getInputStream());
+      // The payload is not serialized if getOutputStream fails.
+      // Calling getInputStream will issue the put/post request with no payload
+      // which causes HTTP 500 server error.
+      if (os == null) {
+        conn.disconnect();
+      } else {
+        IOUtils.closeStream(conn.getInputStream());
+      }
       throw ex;
     }
     if ((conn.getResponseCode() == HttpURLConnection.HTTP_FORBIDDEN


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/35] hadoop git commit: YARN-8194. Fixed reinitialization error for LinuxContainerExecutor. Contributed by Chandni Singh

Posted by xy...@apache.org.
YARN-8194.  Fixed reinitialization error for LinuxContainerExecutor.
            Contributed by Chandni Singh


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f4d280f0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f4d280f0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f4d280f0

Branch: refs/heads/HDDS-4
Commit: f4d280f02b557885cd5e5cf36abc36eb579ccfb4
Parents: 6b63a0a
Author: Eric Yang <ey...@apache.org>
Authored: Wed May 2 20:07:19 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Wed May 2 20:07:19 2018 -0400

----------------------------------------------------------------------
 .../launcher/ContainerLaunch.java               | 37 ++++++++++++++++++++
 .../launcher/ContainerRelaunch.java             | 36 ++-----------------
 2 files changed, 39 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4d280f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
index 9efe686..fa77899 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher;
 
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
+
+import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -844,6 +846,7 @@ public class ContainerLaunch implements Callable<Integer> {
       throw new IOException("Reap container failed for container "
           + containerIdStr);
     }
+    cleanupContainerFiles(getContainerWorkDir());
   }
 
   /**
@@ -1858,4 +1861,38 @@ public class ContainerLaunch implements Callable<Integer> {
       context.getNMStateStore().storeContainerWorkDir(containerId, workDir);
     }
   }
+
+  protected Path getContainerWorkDir() throws IOException {
+    String containerWorkDir = container.getWorkDir();
+    if (containerWorkDir == null
+        || !dirsHandler.isGoodLocalDir(containerWorkDir)) {
+      throw new IOException(
+          "Could not find a good work dir " + containerWorkDir
+              + " for container " + container);
+    }
+
+    return new Path(containerWorkDir);
+  }
+
+  /**
+   * Clean up container's files for container relaunch or cleanup.
+   */
+  protected void cleanupContainerFiles(Path containerWorkDir) {
+    LOG.debug("cleanup container {} files", containerWorkDir);
+    // delete ContainerScriptPath
+    deleteAsUser(new Path(containerWorkDir, CONTAINER_SCRIPT));
+    // delete TokensPath
+    deleteAsUser(new Path(containerWorkDir, FINAL_CONTAINER_TOKENS_FILE));
+  }
+
+  private void deleteAsUser(Path path) {
+    try {
+      exec.deleteAsUser(new DeletionAsUserContext.Builder()
+          .setUser(container.getUser())
+          .setSubDir(path)
+          .build());
+    } catch (Exception e) {
+      LOG.warn("Failed to delete " + path, e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4d280f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java
index c6e3ed4..f69cf96 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerRelaunch.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
-import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,7 +70,8 @@ public class ContainerRelaunch extends ContainerLaunch {
     Path containerLogDir;
     try {
       Path containerWorkDir = getContainerWorkDir();
-      cleanupPreviousContainerFiles(containerWorkDir);
+      // Clean up container's previous files for container relaunch.
+      cleanupContainerFiles(containerWorkDir);
 
       containerLogDir = getContainerLogDir();
 
@@ -148,17 +148,6 @@ public class ContainerRelaunch extends ContainerLaunch {
     return ret;
   }
 
-  private Path getContainerWorkDir() throws IOException {
-    String containerWorkDir = container.getWorkDir();
-    if (containerWorkDir == null
-        || !dirsHandler.isGoodLocalDir(containerWorkDir)) {
-      throw new IOException(
-          "Could not find a good work dir " + containerWorkDir
-          + " for container " + container);
-    }
-
-    return new Path(containerWorkDir);
-  }
 
   private Path getContainerLogDir() throws IOException {
     String containerLogDir = container.getLogDir();
@@ -190,25 +179,4 @@ public class ContainerRelaunch extends ContainerLaunch {
     return dirsHandler.getLocalPathForRead(
         getPidFileSubpath(appIdStr, containerIdStr));
   }
-
-  /**
-   * Clean up container's previous files for container relaunch.
-   */
-  private void cleanupPreviousContainerFiles(Path containerWorkDir) {
-    // delete ContainerScriptPath
-    deleteAsUser(new Path(containerWorkDir, CONTAINER_SCRIPT));
-    // delete TokensPath
-    deleteAsUser(new Path(containerWorkDir, FINAL_CONTAINER_TOKENS_FILE));
-  }
-
-  private void deleteAsUser(Path path) {
-    try {
-      exec.deleteAsUser(new DeletionAsUserContext.Builder()
-          .setUser(container.getUser())
-          .setSubDir(path)
-          .build());
-    } catch (Exception e) {
-      LOG.warn("Failed to delete " + path, e);
-    }
-  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/35] hadoop git commit: YARN-8217. RmAuthenticationFilterInitializer and TimelineAuthenticationFilterInitializer should use Configuration.getPropsWithPrefix instead of iterator. Contributed by Suma Shivaprasad.

Posted by xy...@apache.org.
YARN-8217. RmAuthenticationFilterInitializer and TimelineAuthenticationFilterInitializer should use Configuration.getPropsWithPrefix instead of iterator. Contributed by Suma Shivaprasad.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ee2ce923
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ee2ce923
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ee2ce923

Branch: refs/heads/HDDS-4
Commit: ee2ce923a922bfc3e89ad6f0f6a25e776fe91ffb
Parents: 85381c7
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Thu May 3 10:01:02 2018 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Thu May 3 14:43:40 2018 +0530

----------------------------------------------------------------------
 .../http/RMAuthenticationFilterInitializer.java | 51 ++----------
 ...TimelineAuthenticationFilterInitializer.java | 47 +++---------
 .../security/TestRMAuthenticationFilter.java    | 81 ++++++++++++++++++++
 3 files changed, 98 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee2ce923/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/http/RMAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/http/RMAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/http/RMAuthenticationFilterInitializer.java
index 9fc1334..d0cde9e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/http/RMAuthenticationFilterInitializer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/security/http/RMAuthenticationFilterInitializer.java
@@ -18,23 +18,13 @@
 
 package org.apache.hadoop.yarn.server.security.http;
 
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.FilterContainer;
 import org.apache.hadoop.http.FilterInitializer;
-import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
-import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.AuthenticationFilterInitializer;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
@@ -43,48 +33,23 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 public class RMAuthenticationFilterInitializer extends FilterInitializer {
 
   String configPrefix;
-  String kerberosPrincipalProperty;
-  String cookiePath;
 
   public RMAuthenticationFilterInitializer() {
     this.configPrefix = "hadoop.http.authentication.";
-    this.kerberosPrincipalProperty = KerberosAuthenticationHandler.PRINCIPAL;
-    this.cookiePath = "/";
   }
 
   protected Map<String, String> createFilterConfig(Configuration conf) {
-    Map<String, String> filterConfig = new HashMap<String, String>();
-
-    // setting the cookie path to root '/' so it is used for all resources.
-    filterConfig.put(AuthenticationFilter.COOKIE_PATH, cookiePath);
+    Map<String, String> filterConfig = AuthenticationFilterInitializer
+        .getFilterConfigMap(conf, configPrefix);
 
     // Before conf object is passed in, RM has already processed it and used RM
     // specific configs to overwrite hadoop common ones. Hence we just need to
     // source hadoop.proxyuser configs here.
-    for (Map.Entry<String, String> entry : conf) {
-      String propName = entry.getKey();
-      if (propName.startsWith(configPrefix)) {
-        String value = conf.get(propName);
-        String name = propName.substring(configPrefix.length());
-        filterConfig.put(name, value);
-      } else if (propName.startsWith(ProxyUsers.CONF_HADOOP_PROXYUSER)) {
-        String value = conf.get(propName);
-        String name = propName.substring("hadoop.".length());
-        filterConfig.put(name, value);
-      }
-    }
 
-    // Resolve _HOST into bind address
-    String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
-    String principal = filterConfig.get(kerberosPrincipalProperty);
-    if (principal != null) {
-      try {
-        principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
-      } catch (IOException ex) {
-        throw new RuntimeException(
-          "Could not resolve Kerberos principal name: " + ex.toString(), ex);
-      }
-      filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL, principal);
+    //Add proxy user configs
+    for (Map.Entry<String, String> entry : conf.
+        getPropsWithPrefix(ProxyUsers.CONF_HADOOP_PROXYUSER).entrySet()) {
+      filterConfig.put("proxyuser" + entry.getKey(), entry.getValue());
     }
 
     filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND,
@@ -95,10 +60,8 @@ public class RMAuthenticationFilterInitializer extends FilterInitializer {
 
   @Override
   public void initFilter(FilterContainer container, Configuration conf) {
-
     Map<String, String> filterConfig = createFilterConfig(conf);
     container.addFilter("RMAuthenticationFilter",
       RMAuthenticationFilter.class.getName(), filterConfig);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee2ce923/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
index 3d8ce05..96c3cdf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
@@ -22,8 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.FilterContainer;
 import org.apache.hadoop.http.FilterInitializer;
-import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.AuthenticationFilterInitializer;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
@@ -33,7 +32,6 @@ import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAu
 import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticationHandler;
 import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -62,42 +60,17 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
   protected void setAuthFilterConfig(Configuration conf) {
     filterConfig = new HashMap<String, String>();
 
-    // setting the cookie path to root '/' so it is used for all resources.
-    filterConfig.put(AuthenticationFilter.COOKIE_PATH, "/");
-
-    for (Map.Entry<String, String> entry : conf) {
-      String name = entry.getKey();
-      if (name.startsWith(ProxyUsers.CONF_HADOOP_PROXYUSER)) {
-        String value = conf.get(name);
-        name = name.substring("hadoop.".length());
-        filterConfig.put(name, value);
-      }
-    }
-    for (Map.Entry<String, String> entry : conf) {
-      String name = entry.getKey();
-      if (name.startsWith(PREFIX)) {
-        // yarn.timeline-service.http-authentication.proxyuser will override
-        // hadoop.proxyuser
-        String value = conf.get(name);
-        name = name.substring(PREFIX.length());
-        filterConfig.put(name, value);
-      }
+    for (Map.Entry<String, String> entry : conf
+        .getPropsWithPrefix(ProxyUsers.CONF_HADOOP_PROXYUSER).entrySet()) {
+      filterConfig.put("proxyuser" + entry.getKey(), entry.getValue());
     }
 
-    // Resolve _HOST into bind address
-    String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
-    String principal =
-        filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
-    if (principal != null) {
-      try {
-        principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
-      } catch (IOException ex) {
-        throw new RuntimeException("Could not resolve Kerberos principal " +
-            "name: " + ex.toString(), ex);
-      }
-      filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
-          principal);
-    }
+    // yarn.timeline-service.http-authentication.proxyuser will override
+    // hadoop.proxyuser
+    Map<String, String> timelineAuthProps =
+        AuthenticationFilterInitializer.getFilterConfigMap(conf, PREFIX);
+
+    filterConfig.putAll(timelineAuthProps);
   }
 
   protected Map<String, String> getFilterConfig() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee2ce923/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMAuthenticationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMAuthenticationFilter.java
new file mode 100644
index 0000000..4190cc6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMAuthenticationFilter.java
@@ -0,0 +1,81 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.yarn.server.resourcemanager.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilter;
+import org.apache.hadoop.yarn.server.security.http
+    .RMAuthenticationFilterInitializer;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Test RM Auth filter.
+ */
+public class TestRMAuthenticationFilter {
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testConfiguration() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hadoop.http.authentication.foo", "bar");
+    conf.set("hadoop.proxyuser.user.foo", "bar1");
+
+    conf.set(HttpServer2.BIND_ADDRESS, "barhost");
+
+    FilterContainer container = Mockito.mock(FilterContainer.class);
+    Mockito.doAnswer(new Answer() {
+      @Override
+      public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+        Object[] args = invocationOnMock.getArguments();
+
+        assertEquals("RMAuthenticationFilter", args[0]);
+
+        assertEquals(RMAuthenticationFilter.class.getName(), args[1]);
+
+        Map<String, String> conf = (Map<String, String>) args[2];
+        assertEquals("/", conf.get("cookie.path"));
+
+        assertEquals("simple", conf.get("type"));
+        assertEquals("36000", conf.get("token.validity"));
+        assertNull(conf.get("cookie.domain"));
+        assertEquals("true", conf.get("simple.anonymous.allowed"));
+        assertEquals("HTTP/barhost@LOCALHOST", conf.get("kerberos.principal"));
+        assertEquals(System.getProperty("user.home") + "/hadoop.keytab",
+            conf.get("kerberos.keytab"));
+        assertEquals("bar", conf.get("foo"));
+        assertEquals("bar1", conf.get("proxyuser.user.foo"));
+
+        return null;
+      }
+    }).when(container).addFilter(Mockito.<String>anyObject(),
+        Mockito.<String>anyObject(), Mockito.<Map<String, String>>anyObject());
+
+    new RMAuthenticationFilterInitializer().initFilter(container, conf);
+  }
+}
+


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/35] hadoop git commit: HADOOP-15434. Upgrade to ADLS SDK that exposes current timeout.

Posted by xy...@apache.org.
HADOOP-15434. Upgrade to ADLS SDK that exposes current timeout.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/85381c7b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/85381c7b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/85381c7b

Branch: refs/heads/HDDS-4
Commit: 85381c7b605b5f49664f101cf025e443c300b94c
Parents: e6a80e4
Author: Sean Mackrory <ma...@apache.org>
Authored: Tue May 1 09:47:52 2018 -0600
Committer: Sean Mackrory <ma...@apache.org>
Committed: Wed May 2 21:30:31 2018 -0600

----------------------------------------------------------------------
 hadoop-tools/hadoop-azure-datalake/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/85381c7b/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index 57515b0..5603db9 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -33,7 +33,7 @@
     <minimalJsonVersion>0.9.1</minimalJsonVersion>
     <file.encoding>UTF-8</file.encoding>
     <downloadSources>true</downloadSources>
-    <azure.data.lake.store.sdk.version>2.2.7</azure.data.lake.store.sdk.version>
+    <azure.data.lake.store.sdk.version>2.2.9</azure.data.lake.store.sdk.version>
   </properties>
   <build>
     <plugins>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/35] hadoop git commit: HDFS-5926 Documentation should clarify dfs.datanode.du.reserved impact from reserved disk capacity (Gabor Bota)

Posted by xy...@apache.org.
HDFS-5926 Documentation should clarify dfs.datanode.du.reserved impact from reserved disk capacity (Gabor Bota)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a732acd8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a732acd8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a732acd8

Branch: refs/heads/HDDS-4
Commit: a732acd8730277df4d9b97b97101bc2bc768800f
Parents: 6a69239
Author: Aaron Fabbri <fa...@apache.org>
Authored: Fri May 4 16:37:47 2018 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Fri May 4 16:38:30 2018 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml               | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a732acd8/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index c64b2f1..7a437ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -349,6 +349,9 @@
       For example, reserved space for RAM_DISK storage can be configured using property
       'dfs.datanode.du.reserved.ram_disk'. If specific storage type reservation is not configured
       then dfs.datanode.du.reserved will be used.
+      Note: In case of using tune2fs to set reserved-blocks-percentage, or other filesystem tools,
+      then you can possibly run into out of disk errors because hadoop will not check those
+      external tool configurations.
   </description>
 </property>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/35] hadoop git commit: YARN-7961. Improve status message for YARN service. Contributed by Gour Saha

Posted by xy...@apache.org.
YARN-7961. Improve status message for YARN service.
           Contributed by Gour Saha


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7fe3214d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7fe3214d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7fe3214d

Branch: refs/heads/HDDS-4
Commit: 7fe3214d4bb810c0da18dd936875b4e2588ba518
Parents: ee2ce92
Author: Eric Yang <ey...@apache.org>
Authored: Thu May 3 13:27:07 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu May 3 13:27:07 2018 -0400

----------------------------------------------------------------------
 .../yarn/service/client/ApiServiceClient.java    |  7 +++++++
 .../hadoop/yarn/service/webapp/ApiServer.java    | 10 +++++++---
 .../hadoop/yarn/service/ServiceClientTest.java   |  7 ++++---
 .../hadoop/yarn/service/TestApiServer.java       | 15 ++++++++++++++-
 .../service/client/TestApiServiceClient.java     | 19 ++++++++++++++++++-
 5 files changed, 50 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe3214d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
index cdba555..757e664 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
@@ -479,6 +479,13 @@ public class ApiServiceClient extends AppAdminClient {
     try {
       ClientResponse response = getApiClient(getServicePath(appName))
           .get(ClientResponse.class);
+      if (response.getStatus() == 404) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(" Service ");
+        sb.append(appName);
+        sb.append(" not found");
+        return sb.toString();
+      }
       if (response.getStatus() != 200) {
         StringBuilder sb = new StringBuilder();
         sb.append(appName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe3214d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
index 9a30fcf..8c7c0ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
@@ -186,7 +186,7 @@ public class ApiServer {
     ServiceStatus serviceStatus = new ServiceStatus();
     try {
       if (appName == null) {
-        throw new IllegalArgumentException("Service name can not be null.");
+        throw new IllegalArgumentException("Service name cannot be null.");
       }
       UserGroupInformation ugi = getProxyUser(request);
       LOG.info("GET: getService for appName = {} user = {}", appName, ugi);
@@ -194,12 +194,16 @@ public class ApiServer {
       return Response.ok(app).build();
     } catch (AccessControlException e) {
       return formatResponse(Status.FORBIDDEN, e.getMessage());
-    } catch (IllegalArgumentException |
-        FileNotFoundException e) {
+    } catch (IllegalArgumentException e) {
       serviceStatus.setDiagnostics(e.getMessage());
       serviceStatus.setCode(ERROR_CODE_APP_NAME_INVALID);
       return Response.status(Status.NOT_FOUND).entity(serviceStatus)
           .build();
+    } catch (FileNotFoundException e) {
+      serviceStatus.setDiagnostics("Service " + appName + " not found");
+      serviceStatus.setCode(ERROR_CODE_APP_NAME_INVALID);
+      return Response.status(Status.NOT_FOUND).entity(serviceStatus)
+          .build();
     } catch (IOException | InterruptedException e) {
       LOG.error("Get service failed: {}", e);
       return formatResponse(Status.INTERNAL_SERVER_ERROR, e.getMessage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe3214d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
index cff3e39..73a322c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.service.client.ServiceClient;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
 import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -81,11 +82,11 @@ public class ServiceClientTest extends ServiceClient {
   }
 
   @Override
-  public Service getStatus(String appName) {
-    if (appName != null && appName.equals("jenkins")) {
+  public Service getStatus(String appName) throws FileNotFoundException {
+    if ("jenkins".equals(appName)) {
       return goodServiceStatus;
     } else {
-      throw new IllegalArgumentException();
+      throw new FileNotFoundException("Service " + appName + " not found");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe3214d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java
index 85c3cd4..38aeb59 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.service.api.records.Resource;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.api.records.ServiceStatus;
+import org.apache.hadoop.yarn.service.conf.RestApiConstants;
 import org.apache.hadoop.yarn.service.webapp.ApiServer;
 import org.junit.After;
 import org.junit.Before;
@@ -151,10 +152,17 @@ public class TestApiServer {
 
   @Test
   public void testBadGetService() {
-    final Response actual = apiServer.getService(request, "no-jenkins");
+    final String serviceName = "nonexistent-jenkins";
+    final Response actual = apiServer.getService(request, serviceName);
     assertEquals("Get service is ",
         Response.status(Status.NOT_FOUND).build().getStatus(),
         actual.getStatus());
+    ServiceStatus serviceStatus = (ServiceStatus) actual.getEntity();
+    assertEquals("Response code don't match",
+        RestApiConstants.ERROR_CODE_APP_NAME_INVALID, serviceStatus.getCode());
+    assertEquals("Response diagnostics don't match",
+        "Service " + serviceName + " not found",
+        serviceStatus.getDiagnostics());
   }
 
   @Test
@@ -163,6 +171,11 @@ public class TestApiServer {
     assertEquals("Get service is ",
         Response.status(Status.NOT_FOUND).build().getStatus(),
         actual.getStatus());
+    ServiceStatus serviceStatus = (ServiceStatus) actual.getEntity();
+    assertEquals("Response code don't match",
+        RestApiConstants.ERROR_CODE_APP_NAME_INVALID, serviceStatus.getCode());
+    assertEquals("Response diagnostics don't match",
+        "Service name cannot be null.", serviceStatus.getDiagnostics());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe3214d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java
index a245144..fd31570 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java
@@ -59,7 +59,12 @@ public class TestApiServiceClient {
     protected void doGet(HttpServletRequest req, HttpServletResponse resp)
         throws ServletException, IOException {
       System.out.println("Get was called");
-      resp.setStatus(HttpServletResponse.SC_OK);
+      if (req.getPathInfo() != null
+          && req.getPathInfo().contains("nonexistent-app")) {
+        resp.setStatus(HttpServletResponse.SC_NOT_FOUND);
+      } else {
+        resp.setStatus(HttpServletResponse.SC_OK);
+      }
     }
 
     @Override
@@ -140,6 +145,18 @@ public class TestApiServiceClient {
   }
 
   @Test
+  public void testStatus() {
+    String appName = "nonexistent-app";
+    try {
+      String result = asc.getStatusString(appName);
+      assertEquals("Status reponse don't match",
+          " Service " + appName + " not found", result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
   public void testStop() {
     String appName = "example-app";
     try {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/35] hadoop git commit: HADOOP-12071. conftest is not documented. Contributed by Kengo Seki.

Posted by xy...@apache.org.
HADOOP-12071. conftest is not documented.
Contributed by Kengo Seki.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fe649bb3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fe649bb3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fe649bb3

Branch: refs/heads/HDDS-4
Commit: fe649bb3051f5647073c840d7334a90265ea3f06
Parents: 251f528
Author: Steve Loughran <st...@apache.org>
Authored: Wed May 2 13:33:56 2018 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed May 2 13:33:56 2018 +0100

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/util/ConfTest.java | 10 ++++------
 .../src/site/markdown/CommandsManual.md            | 17 +++++++++++++++++
 2 files changed, 21 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe649bb3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfTest.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfTest.java
index 1915e79..a2cb85f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfTest.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfTest.java
@@ -84,7 +84,7 @@ public final class ConfTest {
     QName property = new QName("property");
 
     List<NodeInfo> nodes = new ArrayList<NodeInfo>();
-    Stack<NodeInfo> parsed = new Stack<NodeInfo>();
+    Stack<NodeInfo> parsed = new Stack<>();
 
     XMLInputFactory factory = XMLInputFactory.newInstance();
     XMLEventReader reader = factory.createXMLEventReader(in);
@@ -258,9 +258,7 @@ public final class ConfTest {
         if (confFile.isFile()) {
           files.add(confFile);
         } else if (confFile.isDirectory()) {
-          for (File file : listFiles(confFile)) {
-            files.add(file);
-          }
+          files.addAll(Arrays.asList(listFiles(confFile)));
         } else {
           terminate(1, confFile.getAbsolutePath()
               + " is neither a file nor directory");
@@ -313,9 +311,9 @@ class NodeInfo {
   private StartElement startElement;
   private List<Attribute> attributes = new ArrayList<Attribute>();
   private Map<StartElement, Characters> elements =
-      new HashMap<StartElement, Characters>();
+      new HashMap<>();
   private Map<QName, List<XMLEvent>> qNameXMLEventsMap =
-      new HashMap<QName, List<XMLEvent>>();
+      new HashMap<>();
 
   public NodeInfo(StartElement startElement) {
     this.startElement = startElement;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe649bb3/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md b/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
index 2839503..ce904c5 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
@@ -99,6 +99,23 @@ Usage: `hadoop classpath [--glob |--jar <path> |-h |--help]`
 
 Prints the class path needed to get the Hadoop jar and the required libraries. If called without arguments, then prints the classpath set up by the command scripts, which is likely to contain wildcards in the classpath entries. Additional options print the classpath after wildcard expansion or write the classpath into the manifest of a jar file. The latter is useful in environments where wildcards cannot be used and the expanded classpath exceeds the maximum supported command line length.
 
+### `conftest`
+
+Usage: `hadoop conftest [-conffile <path>]...`
+
+| COMMAND\_OPTION | Description |
+|:---- |:---- |
+| `-conffile` | Path of a configuration file or directory to validate |
+| `-h`, `--help` | print help |
+
+Validates configuration XML files.
+If the `-conffile` option is not specified, the files in `${HADOOP_CONF_DIR}` whose name end with .xml will be verified. If specified, that path will be verified. You can specify either a file or directory, and if a directory specified, the files in that directory whose name end with `.xml` will be verified.
+You can specify `-conffile` option multiple times.
+
+The validation is fairly minimal: the XML is parsed and duplicate and empty
+property names are checked for. The command does not support XInclude; if you
+using that to pull in configuration items, it will declare the XML file invalid.
+
 ### `credential`
 
 Usage: `hadoop credential <subcommand> [options]`


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/35] hadoop git commit: HADOOP-15444 ITestS3GuardToolDynamo should only run with -Ddynamo (Aaron Fabbri)

Posted by xy...@apache.org.
HADOOP-15444 ITestS3GuardToolDynamo should only run with -Ddynamo (Aaron Fabbri)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/96c843f6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/96c843f6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/96c843f6

Branch: refs/heads/HDDS-4
Commit: 96c843f64bb424cd7544be0ccda16a6755c086de
Parents: 8cdb032
Author: Aaron Fabbri <fa...@apache.org>
Authored: Fri May 4 11:34:37 2018 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Fri May 4 11:34:45 2018 -0700

----------------------------------------------------------------------
 .../hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java     | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c843f6/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
index c7dffd2..821bba5 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import com.amazonaws.services.dynamodbv2.document.DynamoDB;
 import com.amazonaws.services.dynamodbv2.document.Table;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import org.junit.Assume;
 import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
@@ -51,6 +52,14 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
     return new DynamoDBMetadataStore();
   }
 
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    Assume.assumeTrue("Test only applies when DynamoDB is used for S3Guard",
+        getConfiguration().get(Constants.S3_METADATA_STORE_IMPL).equals(
+            Constants.S3GUARD_METASTORE_DYNAMO));
+  }
+
   // Check the existence of a given DynamoDB table.
   private static boolean exist(DynamoDB dynamoDB, String tableName) {
     assertNotNull(dynamoDB);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/35] hadoop git commit: YARN-7818. Remove privileged operation warnings during container launch for the ContainerRuntimes. Contributed by Shane Kumpf

Posted by xy...@apache.org.
YARN-7818. Remove privileged operation warnings during container launch for the ContainerRuntimes. Contributed by Shane Kumpf


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/502914ca
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/502914ca
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/502914ca

Branch: refs/heads/HDDS-4
Commit: 502914ca32ac02b19116fd681eb8301b92fccbb3
Parents: a3b416f
Author: Billie Rinaldi <bi...@apache.org>
Authored: Fri May 4 08:53:55 2018 -0700
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Fri May 4 08:53:55 2018 -0700

----------------------------------------------------------------------
 .../linux/runtime/DefaultLinuxContainerRuntime.java   |  5 +++--
 .../linux/runtime/DockerLinuxContainerRuntime.java    | 14 +++++++-------
 2 files changed, 10 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/502914ca/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DefaultLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DefaultLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DefaultLinuxContainerRuntime.java
index d8db6ad..b5c933a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DefaultLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DefaultLinuxContainerRuntime.java
@@ -108,6 +108,9 @@ public class DefaultLinuxContainerRuntime implements LinuxContainerRuntime {
       launchOp.appendArgs(tcCommandFile);
     }
 
+    // Some failures here are acceptable. Let the calling executor decide.
+    launchOp.disableFailureLogging();
+
     //List<String> -> stored as List -> fetched/converted to List<String>
     //we can't do better here thanks to type-erasure
     @SuppressWarnings("unchecked")
@@ -118,8 +121,6 @@ public class DefaultLinuxContainerRuntime implements LinuxContainerRuntime {
       privilegedOperationExecutor.executePrivilegedOperation(prefixCommands,
             launchOp, null, null, false, false);
     } catch (PrivilegedOperationException e) {
-      LOG.warn("Launch container failed. Exception: ", e);
-
       throw new ContainerExecutionException("Launch container failed", e
           .getExitCode(), e.getOutput(), e.getErrorOutput());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/502914ca/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index ec1d055..0bacd03 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -914,13 +914,13 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     PrivilegedOperation launchOp = buildLaunchOp(ctx,
         commandFile, runCommand);
 
+    // Some failures here are acceptable. Let the calling executor decide.
+    launchOp.disableFailureLogging();
+
     try {
       privilegedOperationExecutor.executePrivilegedOperation(null,
           launchOp, null, null, false, false);
     } catch (PrivilegedOperationException e) {
-      LOG.warn("Launch container failed. Exception: ", e);
-      LOG.info("Docker command used: " + runCommand);
-
       throw new ContainerExecutionException("Launch container failed", e
           .getExitCode(), e.getOutput(), e.getErrorOutput());
     }
@@ -943,14 +943,14 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       PrivilegedOperation launchOp = buildLaunchOp(ctx, commandFile,
           startCommand);
 
+      // Some failures here are acceptable. Let the calling executor decide.
+      launchOp.disableFailureLogging();
+
       try {
         privilegedOperationExecutor.executePrivilegedOperation(null,
             launchOp, null, null, false, false);
       } catch (PrivilegedOperationException e) {
-        LOG.warn("Relaunch container failed. Exception: ", e);
-        LOG.info("Docker command used: " + startCommand);
-
-        throw new ContainerExecutionException("Launch container failed", e
+        throw new ContainerExecutionException("Relaunch container failed", e
             .getExitCode(), e.getOutput(), e.getErrorOutput());
       }
     } else {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/35] hadoop git commit: HADOOP-15406. hadoop-nfs dependencies for mockito and junit are not test scope

Posted by xy...@apache.org.
HADOOP-15406. hadoop-nfs dependencies for mockito and junit are not test scope

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e07156e8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e07156e8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e07156e8

Branch: refs/heads/HDDS-4
Commit: e07156e8b07552b877a22565641465e211144f6f
Parents: 3376872
Author: Jason Lowe <jl...@oath.com>
Authored: Wed May 2 17:30:10 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed May 2 17:30:10 2018 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-nfs/pom.xml | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e07156e8/hadoop-common-project/hadoop-nfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-nfs/pom.xml b/hadoop-common-project/hadoop-nfs/pom.xml
index 8546112..80d8cd2 100644
--- a/hadoop-common-project/hadoop-nfs/pom.xml
+++ b/hadoop-common-project/hadoop-nfs/pom.xml
@@ -56,10 +56,12 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>commons-logging</groupId>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/35] hadoop git commit: HADOOP-15446. WASB: PageBlobInputStream.skip breaks HBASE replication. Contributed by Thomas Marquardt

Posted by xy...@apache.org.
HADOOP-15446. WASB: PageBlobInputStream.skip breaks HBASE replication.
Contributed by Thomas Marquardt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5b11b9fd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5b11b9fd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5b11b9fd

Branch: refs/heads/HDDS-4
Commit: 5b11b9fd413470e134ecdc7c50468f8c7b39fa50
Parents: 67f239c
Author: Steve Loughran <st...@apache.org>
Authored: Mon May 7 11:54:08 2018 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon May 7 11:54:08 2018 +0100

----------------------------------------------------------------------
 .../hadoop/fs/azure/PageBlobInputStream.java    | 123 +++--
 .../fs/azure/ITestPageBlobInputStream.java      | 527 +++++++++++++++++++
 2 files changed, 605 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b11b9fd/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java
index aaac490..40bf6f4 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobInputStream.java
@@ -25,12 +25,14 @@ import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.toShort;
 import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.withMD5Checking;
 
 import java.io.ByteArrayOutputStream;
+import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.azure.StorageInterface.CloudPageBlobWrapper;
 
 import com.microsoft.azure.storage.OperationContext;
@@ -58,7 +60,9 @@ final class PageBlobInputStream extends InputStream {
   // The buffer holding the current data we last read from the server.
   private byte[] currentBuffer;
   // The current byte offset we're at in the buffer.
-  private int currentOffsetInBuffer;
+  private int currentBufferOffset;
+  // The current buffer length
+  private int currentBufferLength;
   // Maximum number of pages to get per any one request.
   private static final int MAX_PAGES_PER_DOWNLOAD =
       4 * 1024 * 1024 / PAGE_SIZE;
@@ -174,7 +178,7 @@ final class PageBlobInputStream extends InputStream {
 
   private boolean dataAvailableInBuffer() {
     return currentBuffer != null 
-        && currentOffsetInBuffer < currentBuffer.length;
+        && currentBufferOffset < currentBufferLength;
   }
 
   /**
@@ -194,6 +198,8 @@ final class PageBlobInputStream extends InputStream {
       return true;
     }
     currentBuffer = null;
+    currentBufferOffset = 0;
+    currentBufferLength = 0;
     if (numberOfPagesRemaining == 0) {
       // No more data to read.
       return false;
@@ -209,43 +215,48 @@ final class PageBlobInputStream extends InputStream {
       ByteArrayOutputStream baos = new ByteArrayOutputStream(bufferSize);
       blob.downloadRange(currentOffsetInBlob, bufferSize, baos,
           withMD5Checking(), opContext);
-      currentBuffer = baos.toByteArray();
+      validateDataIntegrity(baos.toByteArray());
     } catch (StorageException e) {
       throw new IOException(e);
     }
     numberOfPagesRemaining -= pagesToRead;
     currentOffsetInBlob += bufferSize;
-    currentOffsetInBuffer = PAGE_HEADER_SIZE;
-
-    // Since we just downloaded a new buffer, validate its consistency.
-    validateCurrentBufferConsistency();
 
     return true;
   }
 
-  private void validateCurrentBufferConsistency()
+  private void validateDataIntegrity(byte[] buffer)
       throws IOException {
-    if (currentBuffer.length % PAGE_SIZE != 0) {
+
+    if (buffer.length % PAGE_SIZE != 0) {
       throw new AssertionError("Unexpected buffer size: " 
-      + currentBuffer.length);
+      + buffer.length);
     }
-    int numberOfPages = currentBuffer.length / PAGE_SIZE;
+
+    int bufferLength = 0;
+    int numberOfPages = buffer.length / PAGE_SIZE;
+    long totalPagesAfterCurrent = numberOfPagesRemaining;
+
     for (int page = 0; page < numberOfPages; page++) {
-      short currentPageSize = getPageSize(blob, currentBuffer,
-          page * PAGE_SIZE);
-      // Calculate the number of pages that exist after this one
-      // in the blob.
-      long totalPagesAfterCurrent =
-          (numberOfPages - page - 1) + numberOfPagesRemaining;
-      // Only the last page is allowed to be not filled completely.
-      if (currentPageSize < PAGE_DATA_SIZE 
+      // Calculate the number of pages that exist in the blob after this one
+      totalPagesAfterCurrent--;
+
+      short currentPageSize = getPageSize(blob, buffer, page * PAGE_SIZE);
+
+      // Only the last page can be partially filled.
+      if (currentPageSize < PAGE_DATA_SIZE
           && totalPagesAfterCurrent > 0) {
         throw fileCorruptException(blob, String.format(
-            "Page with partial data found in the middle (%d pages from the" 
-            + " end) that only has %d bytes of data.",
-            totalPagesAfterCurrent, currentPageSize));
+            "Page with partial data found in the middle (%d pages from the"
+             + " end) that only has %d bytes of data.",
+             totalPagesAfterCurrent, currentPageSize));
       }
+      bufferLength += currentPageSize + PAGE_HEADER_SIZE;
     }
+
+    currentBufferOffset = PAGE_HEADER_SIZE;
+    currentBufferLength = bufferLength;
+    currentBuffer = buffer;
   }
 
   // Reads the page size from the page header at the given offset.
@@ -275,7 +286,7 @@ final class PageBlobInputStream extends InputStream {
       }
       int bytesRemainingInCurrentPage = getBytesRemainingInCurrentPage();
       int numBytesToRead = Math.min(len, bytesRemainingInCurrentPage);
-      System.arraycopy(currentBuffer, currentOffsetInBuffer, outputBuffer,
+      System.arraycopy(currentBuffer, currentBufferOffset, outputBuffer,
           offset, numBytesToRead);
       numberOfBytesRead += numBytesToRead;
       offset += numBytesToRead;
@@ -284,7 +295,7 @@ final class PageBlobInputStream extends InputStream {
         // We've finished this page, move on to the next.
         advancePagesInBuffer(1);
       } else {
-        currentOffsetInBuffer += numBytesToRead;
+        currentBufferOffset += numBytesToRead;
       }
     }
 
@@ -309,9 +320,26 @@ final class PageBlobInputStream extends InputStream {
   }
 
   /**
-   * Skips over and discards n bytes of data from this input stream.
-   * @param n the number of bytes to be skipped.
-   * @return the actual number of bytes skipped.
+   * Skips over and discards <code>n</code> bytes of data from this input
+   * stream. The <code>skip</code> method may, for a variety of reasons, end
+   * up skipping over some smaller number of bytes, possibly <code>0</code>.
+   * This may result from any of a number of conditions; reaching end of file
+   * before <code>n</code> bytes have been skipped is only one possibility.
+   * The actual number of bytes skipped is returned. If {@code n} is
+   * negative, the {@code skip} method for class {@code InputStream} always
+   * returns 0, and no bytes are skipped. Subclasses may handle the negative
+   * value differently.
+   *
+   * <p> The <code>skip</code> method of this class creates a
+   * byte array and then repeatedly reads into it until <code>n</code> bytes
+   * have been read or the end of the stream has been reached. Subclasses are
+   * encouraged to provide a more efficient implementation of this method.
+   * For instance, the implementation may depend on the ability to seek.
+   *
+   * @param      n   the number of bytes to be skipped.
+   * @return     the actual number of bytes skipped.
+   * @exception  IOException  if the stream does not support seek,
+   *                          or if some other I/O error occurs.
    */
   @Override
   public synchronized long skip(long n) throws IOException {
@@ -338,18 +366,23 @@ final class PageBlobInputStream extends InputStream {
     n -= skippedWithinBuffer;
     long skipped = skippedWithinBuffer;
 
-    // Empty the current buffer, we're going beyond it.
-    currentBuffer = null;
+    if (n == 0) {
+      return skipped;
+    }
+
+    if (numberOfPagesRemaining == 0) {
+      throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+    } else if (numberOfPagesRemaining > 1) {
+      // skip over as many pages as we can, but we must read the last
+      // page as it may not be full
+      long pagesToSkipOver = Math.min(n / PAGE_DATA_SIZE,
+          numberOfPagesRemaining - 1);
+      numberOfPagesRemaining -= pagesToSkipOver;
+      currentOffsetInBlob += pagesToSkipOver * PAGE_SIZE;
+      skipped += pagesToSkipOver * PAGE_DATA_SIZE;
+      n -= pagesToSkipOver * PAGE_DATA_SIZE;
+    }
 
-    // Skip over whole pages as necessary without retrieving them from the
-    // server.
-    long pagesToSkipOver = Math.max(0, Math.min(
-        n / PAGE_DATA_SIZE,
-        numberOfPagesRemaining - 1));
-    numberOfPagesRemaining -= pagesToSkipOver;
-    currentOffsetInBlob += pagesToSkipOver * PAGE_SIZE;
-    skipped += pagesToSkipOver * PAGE_DATA_SIZE;
-    n -= pagesToSkipOver * PAGE_DATA_SIZE;
     if (n == 0) {
       return skipped;
     }
@@ -387,14 +420,14 @@ final class PageBlobInputStream extends InputStream {
 
     // Calculate how many whole pages (pages before the possibly partially
     // filled last page) remain.
-    int currentPageIndex = currentOffsetInBuffer / PAGE_SIZE;
+    int currentPageIndex = currentBufferOffset / PAGE_SIZE;
     int numberOfPagesInBuffer = currentBuffer.length / PAGE_SIZE;
     int wholePagesRemaining = numberOfPagesInBuffer - currentPageIndex - 1;
 
     if (n < (PAGE_DATA_SIZE * wholePagesRemaining)) {
       // I'm within one of the whole pages remaining, skip in there.
       advancePagesInBuffer((int) (n / PAGE_DATA_SIZE));
-      currentOffsetInBuffer += n % PAGE_DATA_SIZE;
+      currentBufferOffset += n % PAGE_DATA_SIZE;
       return n + skipped;
     }
 
@@ -417,8 +450,8 @@ final class PageBlobInputStream extends InputStream {
    */
   private long skipWithinCurrentPage(long n) throws IOException {
     int remainingBytesInCurrentPage = getBytesRemainingInCurrentPage();
-    if (n < remainingBytesInCurrentPage) {
-      currentOffsetInBuffer += n;
+    if (n <= remainingBytesInCurrentPage) {
+      currentBufferOffset += n;
       return n;
     } else {
       advancePagesInBuffer(1);
@@ -438,7 +471,7 @@ final class PageBlobInputStream extends InputStream {
     // Calculate our current position relative to the start of the current
     // page.
     int currentDataOffsetInPage =
-        (currentOffsetInBuffer % PAGE_SIZE) - PAGE_HEADER_SIZE;
+        (currentBufferOffset % PAGE_SIZE) - PAGE_HEADER_SIZE;
     int pageBoundary = getCurrentPageStartInBuffer();
     // Get the data size of the current page from the header.
     short sizeOfCurrentPage = getPageSize(blob, currentBuffer, pageBoundary);
@@ -454,14 +487,14 @@ final class PageBlobInputStream extends InputStream {
   }
 
   private void advancePagesInBuffer(int numberOfPages) {
-    currentOffsetInBuffer =
+    currentBufferOffset =
         getCurrentPageStartInBuffer() 
         + (numberOfPages * PAGE_SIZE) 
         + PAGE_HEADER_SIZE;
   }
 
   private int getCurrentPageStartInBuffer() {
-    return PAGE_SIZE * (currentOffsetInBuffer / PAGE_SIZE);
+    return PAGE_SIZE * (currentBufferOffset / PAGE_SIZE);
   }
 
   private static IOException fileCorruptException(CloudPageBlobWrapper blob,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b11b9fd/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestPageBlobInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestPageBlobInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestPageBlobInputStream.java
new file mode 100644
index 0000000..8c939fc
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestPageBlobInputStream.java
@@ -0,0 +1,527 @@
+/*
+ * 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.azure;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.concurrent.Callable;
+
+import org.junit.FixMethodOrder;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runners.MethodSorters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Test semantics of the page blob input stream
+ */
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+
+public class ITestPageBlobInputStream extends AbstractWasbTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ITestPageBlobInputStream.class);
+  private static final int KILOBYTE = 1024;
+  private static final int MEGABYTE = KILOBYTE * KILOBYTE;
+  private static final int TEST_FILE_SIZE = 6 * MEGABYTE;
+  private static final Path TEST_FILE_PATH = new Path(
+      "TestPageBlobInputStream.txt");
+
+  private long testFileLength;
+
+  /**
+   * Long test timeout.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(10 * 60 * 1000);
+  private FileStatus testFileStatus;
+  private Path hugefile;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    createTestAccount();
+
+    hugefile = fs.makeQualified(TEST_FILE_PATH);
+    try {
+      testFileStatus = fs.getFileStatus(TEST_FILE_PATH);
+      testFileLength = testFileStatus.getLen();
+    } catch (FileNotFoundException e) {
+      // file doesn't exist
+      testFileLength = 0;
+    }
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    Configuration conf = new Configuration();
+
+    // Configure the page blob directories key so every file created is a page blob.
+    conf.set(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES, "/");
+
+    return AzureBlobStorageTestAccount.create(
+        "testpageblobinputstream",
+        EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer),
+        conf,
+        true);
+  }
+
+  /**
+   * Create a test file by repeating the characters in the alphabet.
+   * @throws IOException
+   */
+  private void createTestFileAndSetLength() throws IOException {
+    // To reduce test run time, the test file can be reused.
+    if (fs.exists(TEST_FILE_PATH)) {
+      testFileStatus = fs.getFileStatus(TEST_FILE_PATH);
+      testFileLength = testFileStatus.getLen();
+      LOG.info("Reusing test file: {}", testFileStatus);
+      return;
+    }
+
+    byte[] buffer = new byte[256];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) i;
+    }
+
+    LOG.info("Creating test file {} of size: {}", TEST_FILE_PATH,
+        TEST_FILE_SIZE);
+
+    try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
+      int bytesWritten = 0;
+      while (bytesWritten < TEST_FILE_SIZE) {
+        outputStream.write(buffer);
+        bytesWritten += buffer.length;
+      }
+      LOG.info("Closing stream {}", outputStream);
+      outputStream.close();
+    }
+    testFileLength = fs.getFileStatus(TEST_FILE_PATH).getLen();
+  }
+
+  void assumeHugeFileExists() throws IOException {
+    ContractTestUtils.assertPathExists(fs, "huge file not created", hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
+    ContractTestUtils.assertIsFile(hugefile, status);
+    assertTrue("File " + hugefile + " is empty", status.getLen() > 0);
+  }
+
+  @Test
+  public void test_0100_CreateHugeFile() throws IOException {
+    createTestFileAndSetLength();
+  }
+
+  @Test
+  public void test_0200_BasicReadTest() throws Exception {
+    assumeHugeFileExists();
+
+    try (
+        FSDataInputStream inputStream = fs.open(TEST_FILE_PATH);
+    ) {
+      byte[] buffer = new byte[3 * MEGABYTE];
+
+      // v1 forward seek and read a kilobyte into first kilobyte of buffer
+      long position = 5 * MEGABYTE;
+      inputStream.seek(position);
+      int numBytesRead = inputStream.read(buffer, 0, KILOBYTE);
+      assertEquals(KILOBYTE, numBytesRead);
+
+      byte[] expected = new byte[3 * MEGABYTE];
+
+      for (int i = 0; i < KILOBYTE; i++) {
+        expected[i] = (byte) ((i + position) % 256);
+      }
+
+      assertArrayEquals(expected, buffer);
+
+      int len = MEGABYTE;
+      int offset = buffer.length - len;
+
+      // v1 reverse seek and read a megabyte into last megabyte of buffer
+      position = 3 * MEGABYTE;
+      inputStream.seek(position);
+      numBytesRead = inputStream.read(buffer, offset, len);
+      assertEquals(len, numBytesRead);
+
+      for (int i = offset; i < offset + len; i++) {
+        expected[i] = (byte) ((i + position) % 256);
+      }
+
+      assertArrayEquals(expected, buffer);
+    }
+  }
+
+  @Test
+  public void test_0201_RandomReadTest() throws Exception {
+    assumeHugeFileExists();
+
+    try (
+        FSDataInputStream inputStream = fs.open(TEST_FILE_PATH);
+    ) {
+      final int bufferSize = 4 * KILOBYTE;
+      byte[] buffer = new byte[bufferSize];
+      long position = 0;
+
+      verifyConsistentReads(inputStream, buffer, position);
+
+      inputStream.seek(0);
+
+      verifyConsistentReads(inputStream, buffer, position);
+
+      int seekPosition = 2 * KILOBYTE;
+      inputStream.seek(seekPosition);
+      position = seekPosition;
+      verifyConsistentReads(inputStream, buffer, position);
+
+      inputStream.seek(0);
+      position = 0;
+      verifyConsistentReads(inputStream, buffer, position);
+
+      seekPosition = 5 * KILOBYTE;
+      inputStream.seek(seekPosition);
+      position = seekPosition;
+      verifyConsistentReads(inputStream, buffer, position);
+
+      seekPosition = 10 * KILOBYTE;
+      inputStream.seek(seekPosition);
+      position = seekPosition;
+      verifyConsistentReads(inputStream, buffer, position);
+
+      seekPosition = 4100 * KILOBYTE;
+      inputStream.seek(seekPosition);
+      position = seekPosition;
+      verifyConsistentReads(inputStream, buffer, position);
+
+      for (int i = 4 * 1024 * 1023; i < 5000; i++) {
+        seekPosition = i;
+        inputStream.seek(seekPosition);
+        position = seekPosition;
+        verifyConsistentReads(inputStream, buffer, position);
+      }
+
+      inputStream.seek(0);
+      position = 0;
+      buffer = new byte[1];
+
+      for (int i = 0; i < 5000; i++) {
+        assertEquals(1, inputStream.skip(1));
+        position++;
+        verifyConsistentReads(inputStream, buffer, position);
+        position++;
+      }
+    }
+  }
+
+  private void verifyConsistentReads(FSDataInputStream inputStream,
+                                     byte[] buffer,
+                                     long position) throws IOException {
+    int size = buffer.length;
+    final int numBytesRead = inputStream.read(buffer, 0, size);
+    assertEquals("Bytes read from stream", size, numBytesRead);
+
+    byte[] expected = new byte[size];
+    for (int i = 0; i < expected.length; i++) {
+      expected[i] = (byte) ((position + i) % 256);
+    }
+
+    assertArrayEquals("Mismatch", expected, buffer);
+  }
+
+  /**
+   * Validates the implementation of InputStream.markSupported.
+   * @throws IOException
+   */
+  @Test
+  public void test_0301_MarkSupported() throws IOException {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      assertTrue("mark is not supported", inputStream.markSupported());
+    }
+  }
+
+  /**
+   * Validates the implementation of InputStream.mark and reset
+   * for version 1 of the block blob input stream.
+   * @throws Exception
+   */
+  @Test
+  public void test_0303_MarkAndResetV1() throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      inputStream.mark(KILOBYTE - 1);
+
+      byte[] buffer = new byte[KILOBYTE];
+      int bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+
+      inputStream.reset();
+      assertEquals("rest -> pos 0", 0, inputStream.getPos());
+
+      inputStream.mark(8 * KILOBYTE - 1);
+
+      buffer = new byte[8 * KILOBYTE];
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+
+      intercept(IOException.class,
+          "Resetting to invalid mark",
+          new Callable<FSDataInputStream>() {
+            @Override
+            public FSDataInputStream call() throws Exception {
+              inputStream.reset();
+              return inputStream;
+            }
+          }
+      );
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seekToNewSource, which should
+   * return false for version 1 of the block blob input stream.
+   * @throws IOException
+   */
+  @Test
+  public void test_0305_SeekToNewSourceV1() throws IOException {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      assertFalse(inputStream.seekToNewSource(0));
+    }
+  }
+
+  /**
+   * Validates the implementation of InputStream.skip and ensures there is no
+   * network I/O for version 1 of the block blob input stream.
+   * @throws Exception
+   */
+  @Test
+  public void test_0307_SkipBounds() throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      long skipped = inputStream.skip(-1);
+      assertEquals(0, skipped);
+
+      skipped = inputStream.skip(0);
+      assertEquals(0, skipped);
+
+      assertTrue(testFileLength > 0);
+
+      skipped = inputStream.skip(testFileLength);
+      assertEquals(testFileLength, skipped);
+
+      intercept(EOFException.class,
+          new Callable<Long>() {
+            @Override
+            public Long call() throws Exception {
+              return inputStream.skip(1);
+            }
+          }
+      );
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seek and ensures there is no
+   * network I/O for forward seek.
+   * @throws Exception
+   */
+  @Test
+  public void test_0309_SeekBounds() throws Exception {
+    assumeHugeFileExists();
+    try (
+        FSDataInputStream inputStream = fs.open(TEST_FILE_PATH);
+    ) {
+      inputStream.seek(0);
+      assertEquals(0, inputStream.getPos());
+
+      intercept(EOFException.class,
+          FSExceptionMessages.NEGATIVE_SEEK,
+          new Callable<FSDataInputStream>() {
+            @Override
+            public FSDataInputStream call() throws Exception {
+              inputStream.seek(-1);
+              return inputStream;
+            }
+          }
+      );
+
+      assertTrue("Test file length only " + testFileLength, testFileLength > 0);
+      inputStream.seek(testFileLength);
+      assertEquals(testFileLength, inputStream.getPos());
+
+      intercept(EOFException.class,
+          FSExceptionMessages.CANNOT_SEEK_PAST_EOF,
+          new Callable<FSDataInputStream>() {
+            @Override
+            public FSDataInputStream call() throws Exception {
+              inputStream.seek(testFileLength + 1);
+              return inputStream;
+            }
+          }
+      );
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seek, Seekable.getPos,
+   * and InputStream.available.
+   * @throws Exception
+   */
+  @Test
+  public void test_0311_SeekAndAvailableAndPosition() throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      byte[] expected1 = {0, 1, 2};
+      byte[] expected2 = {3, 4, 5};
+      byte[] expected3 = {1, 2, 3};
+      byte[] expected4 = {6, 7, 8};
+      byte[] buffer = new byte[3];
+
+      int bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected1, buffer);
+      assertEquals(buffer.length, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected2, buffer);
+      assertEquals(2 * buffer.length, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      // reverse seek
+      int seekPos = 0;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected1, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      // reverse seek
+      seekPos = 1;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected3, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      // forward seek
+      seekPos = 6;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected4, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+    }
+  }
+
+  /**
+   * Validates the implementation of InputStream.skip, Seekable.getPos,
+   * and InputStream.available.
+   * @throws IOException
+   */
+  @Test
+  public void test_0313_SkipAndAvailableAndPosition() throws IOException {
+    assumeHugeFileExists();
+    try (
+        FSDataInputStream inputStream = fs.open(TEST_FILE_PATH);
+    ) {
+      byte[] expected1 = {0, 1, 2};
+      byte[] expected2 = {3, 4, 5};
+      byte[] expected3 = {1, 2, 3};
+      byte[] expected4 = {6, 7, 8};
+      assertEquals(testFileLength, inputStream.available());
+      assertEquals(0, inputStream.getPos());
+
+      int n = 3;
+      long skipped = inputStream.skip(n);
+
+      assertEquals(skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+      assertEquals(skipped, n);
+
+      byte[] buffer = new byte[3];
+      int bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected2, buffer);
+      assertEquals(buffer.length + skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      // does skip still work after seek?
+      int seekPos = 1;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected3, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      long currentPosition = inputStream.getPos();
+      n = 2;
+      skipped = inputStream.skip(n);
+
+      assertEquals(currentPosition + skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+      assertEquals(skipped, n);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected4, buffer);
+      assertEquals(buffer.length + skipped + currentPosition,
+          inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+    }
+  }
+
+  @Test
+  public void test_999_DeleteHugeFiles() throws IOException {
+    fs.delete(TEST_FILE_PATH, false);
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/35] hadoop git commit: MAPREDUCE-7086. Add config to allow FileInputFormat to ignore directories when recursive=false. Contributed by Sergey Shelukhin

Posted by xy...@apache.org.
MAPREDUCE-7086. Add config to allow FileInputFormat to ignore directories when recursive=false. Contributed by Sergey Shelukhin


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/68c6ec71
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/68c6ec71
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/68c6ec71

Branch: refs/heads/HDDS-4
Commit: 68c6ec719da8e79ada31c8f3a82124f90b9a71fd
Parents: 24eeea8
Author: Jason Lowe <jl...@apache.org>
Authored: Tue May 1 16:19:53 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue May 1 16:19:53 2018 -0500

----------------------------------------------------------------------
 .../apache/hadoop/mapred/FileInputFormat.java   | 25 ++++++++++++++------
 .../mapreduce/lib/input/FileInputFormat.java    |  8 +++++++
 .../hadoop/mapred/TestFileInputFormat.java      | 17 ++++++++++++-
 .../lib/input/TestFileInputFormat.java          | 12 ++++++++++
 4 files changed, 54 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/68c6ec71/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
index b0ec979..fe43991 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
@@ -78,10 +78,13 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
 
   public static final String NUM_INPUT_FILES =
     org.apache.hadoop.mapreduce.lib.input.FileInputFormat.NUM_INPUT_FILES;
-  
+
   public static final String INPUT_DIR_RECURSIVE = 
     org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR_RECURSIVE;
 
+  public static final String INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS =
+    org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS;
+
 
   private static final double SPLIT_SLOP = 1.1;   // 10% slop
 
@@ -319,16 +322,24 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
   public InputSplit[] getSplits(JobConf job, int numSplits)
     throws IOException {
     StopWatch sw = new StopWatch().start();
-    FileStatus[] files = listStatus(job);
-    
+    FileStatus[] stats = listStatus(job);
+
     // Save the number of input files for metrics/loadgen
-    job.setLong(NUM_INPUT_FILES, files.length);
+    job.setLong(NUM_INPUT_FILES, stats.length);
     long totalSize = 0;                           // compute total size
-    for (FileStatus file: files) {                // check we have valid files
+    boolean ignoreDirs = !job.getBoolean(INPUT_DIR_RECURSIVE, false)
+      && job.getBoolean(INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS, false);
+
+    List<FileStatus> files = new ArrayList<>(stats.length);
+    for (FileStatus file: stats) {                // check we have valid files
       if (file.isDirectory()) {
-        throw new IOException("Not a file: "+ file.getPath());
+        if (!ignoreDirs) {
+          throw new IOException("Not a file: "+ file.getPath());
+        }
+      } else {
+        files.add(file);
+        totalSize += file.getLen();
       }
-      totalSize += file.getLen();
     }
 
     long goalSize = totalSize / (numSplits == 0 ? 1 : numSplits);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68c6ec71/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
index 9868e8e..e2d8e6f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
@@ -76,6 +76,8 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
     "mapreduce.input.fileinputformat.numinputfiles";
   public static final String INPUT_DIR_RECURSIVE =
     "mapreduce.input.fileinputformat.input.dir.recursive";
+  public static final String INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS =
+    "mapreduce.input.fileinputformat.input.dir.nonrecursive.ignore.subdirs";
   public static final String LIST_STATUS_NUM_THREADS =
       "mapreduce.input.fileinputformat.list-status.num-threads";
   public static final int DEFAULT_LIST_STATUS_NUM_THREADS = 1;
@@ -392,7 +394,13 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
     // generate splits
     List<InputSplit> splits = new ArrayList<InputSplit>();
     List<FileStatus> files = listStatus(job);
+
+    boolean ignoreDirs = !getInputDirRecursive(job)
+      && job.getConfiguration().getBoolean(INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS, false);
     for (FileStatus file: files) {
+      if (ignoreDirs && file.isDirectory()) {
+        continue;
+      }
       Path path = file.getPath();
       long length = file.getLen();
       if (length != 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68c6ec71/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileInputFormat.java
index d322011..879cd3d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileInputFormat.java
@@ -102,7 +102,22 @@ public class TestFileInputFormat {
         1, mockFs.numListLocatedStatusCalls);
     FileSystem.closeAll();
   }
-  
+
+  @Test
+  public void testIgnoreDirs() throws Exception {
+    Configuration conf = getConfiguration();
+    conf.setBoolean(FileInputFormat.INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS, true);
+    conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
+    conf.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR, "test:///a1");
+    MockFileSystem mockFs = (MockFileSystem) new Path("test:///").getFileSystem(conf);
+    JobConf job = new JobConf(conf);
+    TextInputFormat fileInputFormat = new TextInputFormat();
+    fileInputFormat.configure(job);
+    InputSplit[] splits = fileInputFormat.getSplits(job, 1);
+    Assert.assertEquals("Input splits are not correct", 1, splits.length);
+    FileSystem.closeAll();
+  }
+
   @Test
   public void testSplitLocationInfo() throws Exception {
     Configuration conf = getConfiguration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68c6ec71/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java
index 4c847fa..3897a9b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java
@@ -124,6 +124,18 @@ public class TestFileInputFormat {
   }
 
   @Test
+  public void testNumInputFilesIgnoreDirs() throws Exception {
+    Configuration conf = getConfiguration();
+    conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
+    conf.setBoolean(FileInputFormat.INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS, true);
+    Job job = Job.getInstance(conf);
+    FileInputFormat<?, ?> fileInputFormat = new TextInputFormat();
+    List<InputSplit> splits = fileInputFormat.getSplits(job);
+    Assert.assertEquals("Input splits are not correct", 1, splits.size());
+    verifySplits(Lists.newArrayList("test:/a1/file1"), splits);
+  }
+
+  @Test
   public void testListLocatedStatus() throws Exception {
     Configuration conf = getConfiguration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/35] hadoop git commit: YARN-8163. Add support for Node Labels in opportunistic scheduling. Contributed by Abhishek Modi.

Posted by xy...@apache.org.
YARN-8163. Add support for Node Labels in opportunistic scheduling. Contributed by Abhishek Modi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6a69239d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6a69239d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6a69239d

Branch: refs/heads/HDDS-4
Commit: 6a69239d867070ee85d79026542033ac661c4c1c
Parents: 4cdbdce
Author: Inigo Goiri <in...@apache.org>
Authored: Fri May 4 14:59:59 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri May 4 14:59:59 2018 -0700

----------------------------------------------------------------------
 .../server/api/protocolrecords/RemoteNode.java  | 40 +++++++++++++++++++-
 .../impl/pb/RemoteNodePBImpl.java               | 19 ++++++++++
 .../OpportunisticContainerAllocator.java        | 38 ++++++++++++++++---
 .../yarn_server_common_service_protos.proto     |  1 +
 .../TestOpportunisticContainerAllocator.java    | 37 ++++++++++++++++++
 ...pportunisticContainerAllocatorAMService.java | 12 ++++++
 ...pportunisticContainerAllocatorAMService.java | 10 ++++-
 7 files changed, 149 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a69239d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java
index f621aa2..67ad5ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RemoteNode.java
@@ -65,6 +65,26 @@ public abstract class RemoteNode implements Comparable<RemoteNode> {
   }
 
   /**
+   * Create new Instance.
+   * @param nodeId NodeId.
+   * @param httpAddress Http address.
+   * @param rackName Rack Name.
+   * @param nodePartition Node Partition.
+   * @return RemoteNode Instance.
+   */
+  @Private
+  @Unstable
+  public static RemoteNode newInstance(NodeId nodeId, String httpAddress,
+      String rackName, String nodePartition) {
+    RemoteNode remoteNode = Records.newRecord(RemoteNode.class);
+    remoteNode.setNodeId(nodeId);
+    remoteNode.setHttpAddress(httpAddress);
+    remoteNode.setRackName(rackName);
+    remoteNode.setNodePartition(nodePartition);
+    return remoteNode;
+  }
+
+  /**
    * Get {@link NodeId}.
    * @return NodeId.
    */
@@ -117,6 +137,23 @@ public abstract class RemoteNode implements Comparable<RemoteNode> {
    * @param other RemoteNode.
    * @return Comparison.
    */
+
+  /**
+   * Get Node Partition.
+   * @return Node Partition.
+   */
+  @Private
+  @Unstable
+  public  abstract String getNodePartition();
+
+  /**
+   * Set Node Partition.
+   * @param nodePartition
+   */
+  @Private
+  @Unstable
+  public abstract void setNodePartition(String nodePartition);
+
   @Override
   public int compareTo(RemoteNode other) {
     return this.getNodeId().compareTo(other.getNodeId());
@@ -127,6 +164,7 @@ public abstract class RemoteNode implements Comparable<RemoteNode> {
     return "RemoteNode{" +
         "nodeId=" + getNodeId() + ", " +
         "rackName=" + getRackName() + ", " +
-        "httpAddress=" + getHttpAddress() + "}";
+        "httpAddress=" + getHttpAddress() + ", " +
+        "partition=" + getNodePartition() + "}";
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a69239d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoteNodePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoteNodePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoteNodePBImpl.java
index c2492cf..8fb4357 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoteNodePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoteNodePBImpl.java
@@ -137,6 +137,25 @@ public class RemoteNodePBImpl extends RemoteNode {
   }
 
   @Override
+  public String getNodePartition() {
+    RemoteNodeProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasNodePartition()) {
+      return null;
+    }
+    return (p.getNodePartition());
+  }
+
+  @Override
+  public void setNodePartition(String nodePartition) {
+    maybeInitBuilder();
+    if (nodePartition == null) {
+      builder.clearNodePartition();
+      return;
+    }
+    builder.setNodePartition(nodePartition);
+  }
+
+  @Override
   public int hashCode() {
     return getProto().hashCode();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a69239d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
index 1f53648..ae1ba43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.scheduler;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -461,10 +462,17 @@ public class OpportunisticContainerAllocator {
   private Collection<RemoteNode> findNodeCandidates(int loopIndex,
       Map<String, RemoteNode> allNodes, Set<String> blackList,
       EnrichedResourceRequest enrichedRR) {
+    LinkedList<RemoteNode> retList = new LinkedList<>();
+    String partition = getRequestPartition(enrichedRR);
     if (loopIndex > 1) {
-      return allNodes.values();
+      for (RemoteNode remoteNode : allNodes.values()) {
+        if (StringUtils.equals(partition, getRemoteNodePartition(remoteNode))) {
+          retList.add(remoteNode);
+        }
+      }
+      return retList;
     } else {
-      LinkedList<RemoteNode> retList = new LinkedList<>();
+
       int numContainers = enrichedRR.getRequest().getNumContainers();
       while (numContainers > 0) {
         if (loopIndex == 0) {
@@ -489,8 +497,10 @@ public class OpportunisticContainerAllocator {
   private int collectRackLocalCandidates(Map<String, RemoteNode> allNodes,
       EnrichedResourceRequest enrichedRR, LinkedList<RemoteNode> retList,
       Set<String> blackList, int numContainers) {
+    String partition = getRequestPartition(enrichedRR);
     for (RemoteNode rNode : allNodes.values()) {
-      if (enrichedRR.getRackLocations().contains(rNode.getRackName())) {
+      if (StringUtils.equals(partition, getRemoteNodePartition(rNode)) &&
+          enrichedRR.getRackLocations().contains(rNode.getRackName())) {
         if (blackList.contains(rNode.getNodeId().getHost())) {
           retList.addLast(rNode);
         } else {
@@ -508,9 +518,11 @@ public class OpportunisticContainerAllocator {
   private int collectNodeLocalCandidates(Map<String, RemoteNode> allNodes,
       EnrichedResourceRequest enrichedRR, List<RemoteNode> retList,
       int numContainers) {
+    String partition = getRequestPartition(enrichedRR);
     for (String nodeName : enrichedRR.getNodeLocations()) {
       RemoteNode remoteNode = allNodes.get(nodeName);
-      if (remoteNode != null) {
+      if (remoteNode != null &&
+          StringUtils.equals(partition, getRemoteNodePartition(remoteNode))) {
         retList.add(remoteNode);
         numContainers--;
       }
@@ -563,7 +575,7 @@ public class OpportunisticContainerAllocator {
             capability, currTime + tokenExpiry,
             tokenSecretManager.getCurrentKey().getKeyId(), rmIdentifier,
             schedulerKey.getPriority(), currTime,
-            null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
+            null, getRemoteNodePartition(node), ContainerType.TASK,
             ExecutionType.OPPORTUNISTIC, schedulerKey.getAllocationRequestId());
     byte[] pwd =
         tokenSecretManager.createPassword(containerTokenIdentifier);
@@ -616,4 +628,20 @@ public class OpportunisticContainerAllocator {
     }
     return partitionedRequests;
   }
+
+  private String getRequestPartition(EnrichedResourceRequest enrichedRR) {
+    String partition = enrichedRR.getRequest().getNodeLabelExpression();
+    if (partition == null) {
+      partition = CommonNodeLabelsManager.NO_LABEL;
+    }
+    return partition;
+  }
+
+  private String getRemoteNodePartition(RemoteNode node) {
+    String partition = node.getNodePartition();
+    if (partition == null) {
+      partition = CommonNodeLabelsManager.NO_LABEL;
+    }
+    return partition;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a69239d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index 1b090bf..387ddb4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -31,6 +31,7 @@ message RemoteNodeProto {
   optional NodeIdProto node_id = 1;
   optional string http_address = 2;
   optional string rack_name = 3;
+  optional string node_partition = 4;
 }
 
 message RegisterDistributedSchedulingAMResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a69239d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java
index 788b0b3..2d3b099 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/scheduler/TestOpportunisticContainerAllocator.java
@@ -596,4 +596,41 @@ public class TestOpportunisticContainerAllocator {
     }
     Assert.assertEquals(100, containers.size());
   }
+
+  @Test
+  public void testAllocationWithNodeLabels() throws Exception {
+    ResourceBlacklistRequest blacklistRequest =
+        ResourceBlacklistRequest.newInstance(
+            new ArrayList<>(), new ArrayList<>());
+    List<ResourceRequest> reqs =
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1),
+            "*", Resources.createResource(1 * GB), 1, true, "label",
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true)));
+    ApplicationAttemptId appAttId = ApplicationAttemptId.newInstance(
+        ApplicationId.newInstance(0L, 1), 1);
+
+    oppCntxt.updateNodeList(
+        Arrays.asList(
+            RemoteNode.newInstance(
+                NodeId.newInstance("h1", 1234), "h1:1234", "/r1")));
+    List<Container> containers = allocator.allocateContainers(
+        blacklistRequest, reqs, appAttId, oppCntxt, 1L, "luser");
+    /* Since there is no node satisfying node label constraints, requests
+       won't get fulfilled.
+    */
+    Assert.assertEquals(0, containers.size());
+    Assert.assertEquals(1, oppCntxt.getOutstandingOpReqs().size());
+
+    oppCntxt.updateNodeList(
+        Arrays.asList(
+            RemoteNode.newInstance(
+                NodeId.newInstance("h1", 1234), "h1:1234", "/r1",
+                "label")));
+
+    containers = allocator.allocateContainers(
+        blacklistRequest, reqs, appAttId, oppCntxt, 1L, "luser");
+    Assert.assertEquals(1, containers.size());
+    Assert.assertEquals(0, oppCntxt.getOutstandingOpReqs().size());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a69239d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
index ce425df..9b13627 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRespons
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
@@ -174,6 +175,16 @@ public class OpportunisticContainerAllocatorAMService
           appAttempt.getOpportunisticContainerContext();
       oppCtx.updateNodeList(getLeastLoadedNodes());
 
+      if (!partitionedAsks.getOpportunistic().isEmpty()) {
+        String appPartition = appAttempt.getAppAMNodePartitionName();
+
+        for (ResourceRequest req : partitionedAsks.getOpportunistic()) {
+          if (null == req.getNodeLabelExpression()) {
+            req.setNodeLabelExpression(appPartition);
+          }
+        }
+      }
+
       List<Container> oppContainers =
           oppContainerAllocator.allocateContainers(
               request.getResourceBlacklistRequest(),
@@ -436,6 +447,7 @@ public class OpportunisticContainerAllocatorAMService
     if (node != null) {
       RemoteNode rNode = RemoteNode.newInstance(nodeId, node.getHttpAddress());
       rNode.setRackName(node.getRackName());
+      rNode.setNodePartition(node.getPartition());
       return rNode;
     }
     return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a69239d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
index efa76bc..5542157 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
@@ -927,6 +927,8 @@ public class TestOpportunisticContainerAllocatorAMService {
                 distAllReq.getProto()));
     Assert.assertEquals(
         "h1", dsAllocResp.getNodesForScheduling().get(0).getNodeId().getHost());
+    Assert.assertEquals(
+        "l1", dsAllocResp.getNodesForScheduling().get(1).getNodePartition());
 
     FinishApplicationMasterResponse dsfinishResp =
         new FinishApplicationMasterResponsePBImpl(
@@ -1004,9 +1006,13 @@ public class TestOpportunisticContainerAllocatorAMService {
             .getExecutionTypeRequest().getEnforceExecutionType());
         DistributedSchedulingAllocateResponse resp = factory
             .newRecordInstance(DistributedSchedulingAllocateResponse.class);
+        RemoteNode remoteNode1 = RemoteNode.newInstance(
+            NodeId.newInstance("h1", 1234), "http://h1:4321");
+        RemoteNode remoteNode2 = RemoteNode.newInstance(
+            NodeId.newInstance("h2", 1234), "http://h2:4321");
+        remoteNode2.setNodePartition("l1");
         resp.setNodesForScheduling(
-            Arrays.asList(RemoteNode.newInstance(
-                NodeId.newInstance("h1", 1234), "http://h1:4321")));
+            Arrays.asList(remoteNode1, remoteNode2));
         return resp;
       }
     };


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/35] hadoop git commit: HDDS-15. Add memory profiler support to Genesis. Contributed by Anu Engineer.

Posted by xy...@apache.org.
HDDS-15. Add memory profiler support to Genesis. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6b63a0af
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6b63a0af
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6b63a0af

Branch: refs/heads/HDDS-4
Commit: 6b63a0af9b29c231166d9af50d499a246cbbb755
Parents: 3b34fca
Author: Anu Engineer <ae...@apache.org>
Authored: Wed May 2 10:44:47 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Wed May 2 12:54:49 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ozone/genesis/Genesis.java    |  7 ++-
 .../ozone/genesis/GenesisMemoryProfiler.java    | 59 ++++++++++++++++++++
 2 files changed, 65 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b63a0af/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/Genesis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/Genesis.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/Genesis.java
index 5efa12a..0dc3db7 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/Genesis.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/Genesis.java
@@ -42,9 +42,12 @@ public final class Genesis {
         .include(BenchMarkMetadataStoreReads.class.getSimpleName())
         .include(BenchMarkMetadataStoreWrites.class.getSimpleName())
         .include(BenchMarkDatanodeDispatcher.class.getSimpleName())
-        .include(BenchMarkRocksDbStore.class.getSimpleName())
+// Commenting this test out, till we support either a command line or a config
+        // file based ability to run tests.
+//        .include(BenchMarkRocksDbStore.class.getSimpleName())
         .warmupIterations(5)
         .measurementIterations(20)
+        .addProfiler(GenesisMemoryProfiler.class)
         .shouldDoGC(true)
         .forks(1)
         .build();
@@ -52,3 +55,5 @@ public final class Genesis {
     new Runner(opt).run();
   }
 }
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b63a0af/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisMemoryProfiler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisMemoryProfiler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisMemoryProfiler.java
new file mode 100644
index 0000000..090f1a7
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisMemoryProfiler.java
@@ -0,0 +1,59 @@
+/*
+ * 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.genesis;
+
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.infra.IterationParams;
+import org.openjdk.jmh.profile.InternalProfiler;
+import org.openjdk.jmh.results.AggregationPolicy;
+import org.openjdk.jmh.results.IterationResult;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.ScalarResult;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Max memory profiler.
+ */
+public class GenesisMemoryProfiler implements InternalProfiler {
+  @Override
+  public void beforeIteration(BenchmarkParams benchmarkParams,
+      IterationParams iterationParams) {
+
+  }
+
+  @Override
+  public Collection<? extends Result> afterIteration(BenchmarkParams
+      benchmarkParams, IterationParams iterationParams, IterationResult
+      result) {
+    long totalHeap = Runtime.getRuntime().totalMemory();
+
+    Collection<ScalarResult> samples = new ArrayList<>();
+    samples.add(new ScalarResult("Max heap", totalHeap, "bytes",
+        AggregationPolicy.MAX));
+    return samples;
+  }
+
+  @Override
+  public String getDescription() {
+    return "Genesis Memory Profiler. Computes Max Memory used by a test.";
+  }
+}
+


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/35] hadoop git commit: YARN-8113. Update placement constraints doc with application namespaces and inter-app constraints. Contributed by Weiwei Yang.

Posted by xy...@apache.org.
YARN-8113. Update placement constraints doc with application namespaces and inter-app constraints. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3b34fca4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3b34fca4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3b34fca4

Branch: refs/heads/HDDS-4
Commit: 3b34fca4b5d67a2685852f30bb61e7c408a0e886
Parents: 883f682
Author: Konstantinos Karanasos <kk...@apache.org>
Authored: Wed May 2 11:48:35 2018 -0700
Committer: Konstantinos Karanasos <kk...@apache.org>
Committed: Wed May 2 11:49:56 2018 -0700

----------------------------------------------------------------------
 .../site/markdown/PlacementConstraints.md.vm    | 67 +++++++++++++++-----
 1 file changed, 52 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34fca4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/PlacementConstraints.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/PlacementConstraints.md.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/PlacementConstraints.md.vm
index cb34c3f..4ac1683 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/PlacementConstraints.md.vm
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/PlacementConstraints.md.vm
@@ -28,7 +28,7 @@ YARN allows applications to specify placement constraints in the form of data lo
 
 For example, it may be beneficial to co-locate the allocations of a job on the same rack (*affinity* constraints) to reduce network costs, spread allocations across machines (*anti-affinity* constraints) to minimize resource interference, or allow up to a specific number of allocations in a node group (*cardinality* constraints) to strike a balance between the two. Placement decisions also affect resilience. For example, allocations placed within the same cluster upgrade domain would go offline simultaneously.
 
-The applications can specify constraints without requiring knowledge of the underlying topology of the cluster (e.g., one does not need to specify the specific node or rack where their containers should be placed with constraints) or the other applications deployed. Currently **intra-application** constraints are supported, but the design that is followed is generic and support for constraints across applications will soon be added. Moreover, all constraints at the moment are **hard**, that is, if the constraints for a container cannot be satisfied due to the current cluster condition or conflicting constraints, the container request will remain pending or get will get rejected.
+The applications can specify constraints without requiring knowledge of the underlying topology of the cluster (e.g., one does not need to specify the specific node or rack where their containers should be placed with constraints) or the other applications deployed. Currently, all constraints are **hard**, that is, if a constraint for a container cannot be satisfied due to the current cluster condition or conflicting constraints, the container request will remain pending or get rejected.
 
 Note that in this document we use the notion of “allocation” to refer to a unit of resources (e.g., CPU and memory) that gets allocated in a node. In the current implementation of YARN, an allocation corresponds to a single container. However, in case an application uses an allocation to spawn more than one containers, an allocation could correspond to multiple containers.
 
@@ -65,15 +65,19 @@ $ yarn org.apache.hadoop.yarn.applications.distributedshell.Client -jar share/ha
 where **PlacementSpec** is of the form:
 
 ```
-PlacementSpec => "" | KeyVal;PlacementSpec
-KeyVal        => SourceTag=Constraint
-SourceTag     => String
-Constraint    => NumContainers | NumContainers,"IN",Scope,TargetTag | NumContainers,"NOTIN",Scope,TargetTag | NumContainers,"CARDINALITY",Scope,TargetTag,MinCard,MaxCard
-NumContainers => int
-Scope         => "NODE" | "RACK"
-TargetTag     => String
-MinCard       => int
-MaxCard       => int
+PlacementSpec         => "" | KeyVal;PlacementSpec
+KeyVal                => SourceTag=ConstraintExpr
+SourceTag             => String
+ConstraintExpr        => NumContainers | NumContainers, Constraint
+Constraint            => SingleConstraint | CompositeConstraint
+SingleConstraint      => "IN",Scope,TargetTag | "NOTIN",Scope,TargetTag | "CARDINALITY",Scope,TargetTag,MinCard,MaxCard
+CompositeConstraint   => AND(ConstraintList) | OR(ConstraintList)
+ConstraintList        => Constraint | Constraint:ConstraintList
+NumContainers         => int
+Scope                 => "NODE" | "RACK"
+TargetTag             => String
+MinCard               => int
+MaxCard               => int
 ```
 
 Note that when the `-placement_spec` argument is specified in the distributed shell command, the `-num-containers` argument should not be used. In case `-num-containers` argument is used in conjunction with `-placement-spec`, the former is ignored. This is because in PlacementSpec, we determine the number of containers per tag, making the `-num-containers` redundant and possibly conflicting. Moreover, if `-placement_spec` is used, all containers will be requested with GUARANTEED execution type.
@@ -82,11 +86,18 @@ An example of PlacementSpec is the following:
 ```
 zk=3,NOTIN,NODE,zk:hbase=5,IN,RACK,zk:spark=7,CARDINALITY,NODE,hbase,1,3
 ```
-The above encodes two constraints:
+The above encodes three constraints:
 * place 3 containers with tag "zk" (standing for ZooKeeper) with node anti-affinity to each other, i.e., do not place more than one container per node (notice that in this first constraint, the SourceTag and the TargetTag of the constraint coincide);
 * place 5 containers with tag "hbase" with affinity to a rack on which containers with tag "zk" are running (i.e., an "hbase" container should not be placed at a rack where an "zk" container is running, given that "zk" is the TargetTag of the second constraint);
-* place 7 container with tag "spark" in nodes that have at least one, but no more than three, containers, with tag "hbase".
+* place 7 containers with tag "spark" in nodes that have at least one, but no more than three, containers with tag "hbase".
 
+Another example below demonstrates a composite form of constraint:
+```
+zk=5,AND(IN,RACK,hbase:NOTIN,NODE,zk)
+```
+The above constraint uses the conjunction operator `AND` to combine two constraints. The AND constraint is satisfied when both its children constraints are satisfied. The specific PlacementSpec requests to place 5 "zk" containers in a rack where at least one "hbase" container is running, and on a node that no "zk" container is running.
+Similarly, an `OR` operator can be used to define a constraint that is satisfied when at least one of its children constraints is satisfied.
+Note that in case "zk" and "hbase" are containers belonging to different applications (which is most probably the case in real use cases), the allocation tags in the PlacementSpec should include namespaces, as we describe below (see [Allocation tags namespace](#Allocation_tags_namespace)).
 
 
 Defining Placement Constraints
@@ -98,11 +109,37 @@ Allocation tags are string tags that an application can associate with (groups o
 
 Note that instead of using the `ResourceRequest` object to define allocation tags, we use the new `SchedulingRequest` object. This has many similarities with the `ResourceRequest`, but better separates the sizing of the requested allocations (number and size of allocations, priority, execution type, etc.), and the constraints dictating how these allocations should be placed (resource name, relaxed locality). Applications can still use `ResourceRequest` objects, but in order to define allocation tags and constraints, they need to use the `SchedulingRequest` object. Within a single `AllocateRequest`, an application should use either the `ResourceRequest` or the `SchedulingRequest` objects, but not both of them.
 
+$H4 Allocation tags namespace
+
+Allocation tags might refer to containers of the same or different applications, and are used to express intra- or inter-application constraints, respectively.
+We use allocation tag namespaces in order to specify the scope of applications that an allocation tag can refer to. By coupling an allocation tag with a namespace, we can restrict whether the tag targets containers that belong to the same application, to a certain group of applications, or to any application in the cluster.
+
+We currently support the following namespaces:
+
+| Namespace | Syntax | Description |
+|:--------- |:-------|:------------|
+| SELF | `self/${allocationTag}` | The allocation tag refers to containers of the current application (to which the constraint will be applied). This is the default namespace. |
+| NOT_SELF | `not-self/${allocationTag}` | The allocation tag refers only to containers that do not belong to the current application. |
+| ALL | `all/${allocationTag}` | The allocation tag refers to containers of any application. |
+| APP_ID | `app-id/${applicationID}/${allocationTag}` | The allocation tag refers to containers of the application with the specified application ID. |
+| APP_TAG | `app-tag/application_tag_name/${allocationTag}` | The allocation tag refers to containers of applications that are tagged with the specified application tag. |
+
+
+To attach an allocation tag namespace `ns` to a target tag `targetTag`, we use the syntax `ns/allocationTag` in the PlacementSpec. Note that the default namespace is `SELF`, which is used for **intra-app** constraints. The remaining namespace tags are used to specify **inter-app** constraints. When the namespace is not specified next to a tag, `SELF` is assumed.
+
+The example constraints used above could be extended with namespaces as follows:
+```
+zk=3,NOTIN,NODE,not-self/zk:hbase=5,IN,RACK,all/zk:spark=7,CARDINALITY,NODE,app-id/appID_0023/hbase,1,3
+```
+The semantics of these constraints are the following:
+* place 3 containers with tag "zk" (standing for ZooKeeper) to nodes that do not have "zk" containers from other applications running;
+* place 5 containers with tag "hbase" with affinity to a rack on which containers with tag "zk" (from any application, be it the same or a different one) are running;
+* place 7 containers with tag "spark" in nodes that have at least one, but no more than three, containers with tag "hbase" belonging to application with ID `appID_0023`.
+
 $H4 Differences between node labels, node attributes and allocation tags
 
 The difference between allocation tags and node labels or node attributes (YARN-3409), is that allocation tags are attached to allocations and not to nodes. When an allocation gets allocated to a node by the scheduler, the set of tags of that allocation are automatically added to the node for the duration of the allocation. Hence, a node inherits the tags of the allocations that are currently allocated to the node. Likewise, a rack inherits the tags of its nodes. Moreover, similar to node labels and unlike node attributes, allocation tags have no value attached to them. As we show below, our constraints can refer to allocation tags, as well as node labels and node attributes.
 
-
 $H3 Placement constraints API
 
 Applications can use the public API in the `PlacementConstraints` to construct placement constraint. Before describing the methods for building constraints, we describe the methods of the `PlacementTargets` class that are used to construct the target expressions that will then be used in constraints:
@@ -110,7 +147,7 @@ Applications can use the public API in the `PlacementConstraints` to construct p
 | Method | Description |
 |:------ |:----------- |
 | `allocationTag(String... allocationTags)` | Constructs a target expression on an allocation tag. It is satisfied if there are allocations with one of the given tags. |
-| `allocationTagToIntraApp(String... allocationTags)` | similar to `allocationTag(String...)`, but targeting only the containers of the application that will use this target (intra-application constraints). |
+| `allocationTagWithNamespace(String namespace, String... allocationTags)` | Similar to `allocationTag(String...)`, but allows to specify a namespace for the given allocation tags. |
 | `nodePartition(String... nodePartitions)` | Constructs a target expression on a node partition. It is satisfied for nodes that belong to one of the `nodePartitions`. |
 | `nodeAttribute(String attributeKey, String... attributeValues)` | Constructs a target expression on a node attribute. It is satisfied if the specified node attribute has one of the specified values. |
 
@@ -136,4 +173,4 @@ Applications have to specify the containers for which each constraint will be en
 
 When using the `placement-processor` handler (see [Enabling placement constraints](#Enabling_placement_constraints)), this constraint mapping is specified within the `RegisterApplicationMasterRequest`.
 
-When using the `scheduler` handler, the constraints can also be added at each `SchedulingRequest` object. Each such constraint is valid for the tag of that scheduling request. In case constraints are specified both at the `RegisterApplicationMasterRequest` and the scheduling requests, the latter override the former.
+When using the `scheduler` handler, the constraints can also be added at each `SchedulingRequest` object. Each such constraint is valid for the tag of that scheduling request. In case constraints are specified both at the `RegisterApplicationMasterRequest` and the scheduling requests, the latter override the former.
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/35] hadoop git commit: YARN-8209. Fixed NPE in Yarn Service deletion. Contributed by Eric Badger

Posted by xy...@apache.org.
YARN-8209.  Fixed NPE in Yarn Service deletion.
            Contributed by Eric Badger


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/883f6822
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/883f6822
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/883f6822

Branch: refs/heads/HDDS-4
Commit: 883f68222a9cfd06f79a8fcd75ec9fef00abc035
Parents: 19ae588
Author: Eric Yang <ey...@apache.org>
Authored: Wed May 2 14:33:31 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Wed May 2 14:33:31 2018 -0400

----------------------------------------------------------------------
 .../linux/privileged/PrivilegedOperation.java   |  4 +-
 .../runtime/DockerLinuxContainerRuntime.java    | 21 +++++----
 .../linux/runtime/docker/DockerClient.java      |  7 ++-
 .../linux/runtime/docker/DockerCommand.java     | 32 +++++++++++++
 .../runtime/docker/DockerCommandExecutor.java   | 12 ++---
 .../runtime/docker/DockerInspectCommand.java    | 19 ++++++++
 .../linux/runtime/docker/DockerRmCommand.java   | 16 +++++++
 .../impl/container-executor.c                   | 28 ++++++++++++
 .../impl/container-executor.h                   | 10 ++++-
 .../main/native/container-executor/impl/main.c  | 47 ++++++++++++++++++--
 .../docker/TestDockerCommandExecutor.java       | 31 +++++++------
 11 files changed, 184 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
index 189c0d0..92a82e8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
@@ -54,7 +54,9 @@ public class PrivilegedOperation {
     GPU("--module-gpu"),
     FPGA("--module-fpga"),
     LIST_AS_USER(""), // no CLI switch supported yet.
-    ADD_NUMA_PARAMS(""); // no CLI switch supported yet.
+    ADD_NUMA_PARAMS(""), // no CLI switch supported yet.
+    REMOVE_DOCKER_CONTAINER("--remove-docker-container"),
+    INSPECT_DOCKER_CONTAINER("--inspect-docker-container");
 
     private final String option;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index 9c05c59..ec1d055 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerCommand;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker.DockerCommandExecutor;
@@ -384,7 +385,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       Container container) throws ContainerExecutionException {
     try {
       String commandFile = dockerClient.writeCommandToTempFile(
-          dockerVolumeCommand, container, nmContext);
+          dockerVolumeCommand, container.getContainerId(), nmContext);
       PrivilegedOperation privOp = new PrivilegedOperation(
           PrivilegedOperation.OperationType.RUN_DOCKER_CMD);
       privOp.appendArgs(commandFile);
@@ -734,6 +735,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   public void launchContainer(ContainerRuntimeContext ctx)
       throws ContainerExecutionException {
     Container container = ctx.getContainer();
+    ContainerId containerId = container.getContainerId();
     Map<String, String> environment = container.getLaunchContext()
         .getEnvironment();
     String imageName = environment.get(ENV_DOCKER_CONTAINER_IMAGE);
@@ -750,7 +752,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
 
     validateImageName(imageName);
 
-    String containerIdStr = container.getContainerId().toString();
+    String containerIdStr = containerId.toString();
     String runAsUser = ctx.getExecutionAttribute(RUN_AS_USER);
     String dockerRunAsUser = runAsUser;
     Path containerWorkDir = ctx.getExecutionAttribute(CONTAINER_WORK_DIR);
@@ -908,7 +910,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     }
 
     String commandFile = dockerClient.writeCommandToTempFile(runCommand,
-        container, nmContext);
+        containerId, nmContext);
     PrivilegedOperation launchOp = buildLaunchOp(ctx,
         commandFile, runCommand);
 
@@ -927,8 +929,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   @Override
   public void relaunchContainer(ContainerRuntimeContext ctx)
       throws ContainerExecutionException {
-    Container container = ctx.getContainer();
-    String containerIdStr = container.getContainerId().toString();
+    ContainerId containerId = ctx.getContainer().getContainerId();
+    String containerIdStr = containerId.toString();
     // Check to see if the container already exists for relaunch
     DockerCommandExecutor.DockerContainerStatus containerStatus =
         DockerCommandExecutor.getContainerStatus(containerIdStr, conf,
@@ -937,7 +939,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         DockerCommandExecutor.isStartable(containerStatus)) {
       DockerStartCommand startCommand = new DockerStartCommand(containerIdStr);
       String commandFile = dockerClient.writeCommandToTempFile(startCommand,
-          container, nmContext);
+          containerId, nmContext);
       PrivilegedOperation launchOp = buildLaunchOp(ctx, commandFile,
           startCommand);
 
@@ -1042,12 +1044,13 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   // ipAndHost[1] contains the hostname.
   @Override
   public String[] getIpAndHost(Container container) {
-    String containerId = container.getContainerId().toString();
+    ContainerId containerId = container.getContainerId();
+    String containerIdStr = containerId.toString();
     DockerInspectCommand inspectCommand =
-        new DockerInspectCommand(containerId).getIpAndHost();
+        new DockerInspectCommand(containerIdStr).getIpAndHost();
     try {
       String commandFile = dockerClient.writeCommandToTempFile(inspectCommand,
-          container, nmContext);
+          containerId, nmContext);
       PrivilegedOperation privOp = new PrivilegedOperation(
           PrivilegedOperation.OperationType.RUN_DOCKER_CMD);
       privOp.appendArgs(commandFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerClient.java
index c55b83b..7bd4546 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerClient.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
 import org.slf4j.Logger;
@@ -104,9 +103,9 @@ public final class DockerClient {
     }
   }
 
-  public String writeCommandToTempFile(DockerCommand cmd, Container container,
-      Context nmContext) throws ContainerExecutionException {
-    ContainerId containerId = container.getContainerId();
+  public String writeCommandToTempFile(DockerCommand cmd,
+      ContainerId containerId, Context nmContext)
+      throws ContainerExecutionException {
     String filePrefix = containerId.toString();
     ApplicationId appId = containerId.getApplicationAttemptId()
         .getApplicationId();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommand.java
index 0124c83..366457d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommand.java
@@ -22,7 +22,12 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -104,4 +109,31 @@ public abstract class DockerCommand {
       addCommandArguments("docker-config", clientConfigDir);
     }
   }
+
+  /**
+   * Prepare the privileged operation object that will be used to invoke
+   * the container-executor.
+   *
+   * @param dockerCommand Specific command to be run by docker.
+   * @param containerName
+   * @param env
+   * @param conf
+   * @param nmContext
+   * @return Returns the PrivilegedOperation object to be used.
+   * @throws ContainerExecutionException
+   */
+  public PrivilegedOperation preparePrivilegedOperation(
+      DockerCommand dockerCommand, String containerName, Map<String,
+      String> env, Configuration conf, Context nmContext)
+      throws ContainerExecutionException {
+    DockerClient dockerClient = new DockerClient(conf);
+    String commandFile =
+        dockerClient.writeCommandToTempFile(dockerCommand,
+        ContainerId.fromString(containerName),
+        nmContext);
+    PrivilegedOperation dockerOp = new PrivilegedOperation(
+        PrivilegedOperation.OperationType.RUN_DOCKER_CMD);
+    dockerOp.appendArgs(commandFile);
+    return dockerOp;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommandExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommandExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommandExecutor.java
index 6abe1cb..8a4888c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommandExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerCommandExecutor.java
@@ -17,7 +17,6 @@
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
@@ -80,14 +79,9 @@ public final class DockerCommandExecutor {
       PrivilegedOperationExecutor privilegedOperationExecutor,
       boolean disableFailureLogging, Context nmContext)
       throws ContainerExecutionException {
-    DockerClient dockerClient = new DockerClient(conf);
-    String commandFile =
-        dockerClient.writeCommandToTempFile(dockerCommand,
-        nmContext.getContainers().get(ContainerId.fromString(containerId)),
-        nmContext);
-    PrivilegedOperation dockerOp = new PrivilegedOperation(
-        PrivilegedOperation.OperationType.RUN_DOCKER_CMD);
-    dockerOp.appendArgs(commandFile);
+    PrivilegedOperation dockerOp = dockerCommand.preparePrivilegedOperation(
+        dockerCommand, containerId, env, conf, nmContext);
+
     if (disableFailureLogging) {
       dockerOp.disableFailureLogging();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerInspectCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerInspectCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerInspectCommand.java
index d27f74d0..3ed9c18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerInspectCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerInspectCommand.java
@@ -20,12 +20,19 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+
+import java.util.Map;
+
 /**
  * Encapsulates the docker inspect command and its command
  * line arguments.
  */
 public class DockerInspectCommand extends DockerCommand {
   private static final String INSPECT_COMMAND = "inspect";
+  private String commandArguments;
 
   public DockerInspectCommand(String containerName) {
     super(INSPECT_COMMAND);
@@ -34,6 +41,7 @@ public class DockerInspectCommand extends DockerCommand {
 
   public DockerInspectCommand getContainerStatus() {
     super.addCommandArguments("format", "{{.State.Status}}");
+    this.commandArguments = "--format={{.State.Status}}";
     return this;
   }
 
@@ -43,6 +51,17 @@ public class DockerInspectCommand extends DockerCommand {
     // cannot parse the arguments correctly.
     super.addCommandArguments("format", "{{range(.NetworkSettings.Networks)}}"
         + "{{.IPAddress}},{{end}}{{.Config.Hostname}}");
+    this.commandArguments = "--format={{range(.NetworkSettings.Networks)}}"
+        + "{{.IPAddress}},{{end}}{{.Config.Hostname}}";
     return this;
   }
+  @Override
+  public PrivilegedOperation preparePrivilegedOperation(
+      DockerCommand dockerCommand, String containerName, Map<String,
+      String> env, Configuration conf, Context nmContext) {
+    PrivilegedOperation dockerOp = new PrivilegedOperation(
+        PrivilegedOperation.OperationType.INSPECT_DOCKER_CONTAINER);
+    dockerOp.appendArgs(commandArguments, containerName);
+    return dockerOp;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRmCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRmCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRmCommand.java
index dcfe777..3a02982 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRmCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRmCommand.java
@@ -16,6 +16,12 @@
  */
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.docker;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+
+import java.util.Map;
+
 /**
  * Encapsulates the docker rm command and its command
  * line arguments.
@@ -27,4 +33,14 @@ public class DockerRmCommand extends DockerCommand {
     super(RM_COMMAND);
     super.addCommandArguments("name", containerName);
   }
+
+  @Override
+  public PrivilegedOperation preparePrivilegedOperation(
+      DockerCommand dockerCommand, String containerName, Map<String,
+      String> env, Configuration conf, Context nmContext) {
+    PrivilegedOperation dockerOp = new PrivilegedOperation(
+        PrivilegedOperation.OperationType.REMOVE_DOCKER_CONTAINER);
+    dockerOp.appendArgs(containerName);
+    return dockerOp;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index d9ed070..6b4ec0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -1332,6 +1332,34 @@ int run_docker(const char *command_file) {
   return exit_code;
 }
 
+int exec_docker_command(char *docker_command, char **argv,
+    int argc, int optind) {
+  int i;
+  char* docker_binary = get_docker_binary(&CFG);
+  size_t command_size = argc - optind + 2;
+
+  char **args = alloc_and_clear_memory(command_size + 1, sizeof(char));
+  args[0] = docker_binary;
+  args[1] = docker_command;
+  for(i = 2; i < command_size; i++) {
+    args[i] = (char *) argv[i];
+  }
+  args[i] = NULL;
+
+  execvp(docker_binary, args);
+
+  // will only get here if execvp fails
+  fprintf(ERRORFILE, "Couldn't execute the container launch with args %s - %s\n",
+      docker_binary, strerror(errno));
+  fflush(LOGFILE);
+  fflush(ERRORFILE);
+
+  free(docker_binary);
+  free(args);
+
+  return DOCKER_RUN_FAILED;
+}
+
 int create_script_paths(const char *work_dir,
   const char *script_name, const char *cred_file,
   char** script_file_dest, char** cred_file_dest,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
index 7c3ed77..47c4221 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
@@ -47,7 +47,9 @@ enum operations {
   RUN_AS_USER_DELETE = 9,
   RUN_AS_USER_LAUNCH_DOCKER_CONTAINER = 10,
   RUN_DOCKER = 11,
-  RUN_AS_USER_LIST = 12
+  RUN_AS_USER_LIST = 12,
+  REMOVE_DOCKER_CONTAINER = 13,
+  INSPECT_DOCKER_CONTAINER = 14
 };
 
 #define NM_GROUP_KEY "yarn.nodemanager.linux-container-executor.group"
@@ -263,6 +265,12 @@ int is_docker_support_enabled();
  */
 int run_docker(const char *command_file);
 
+/**
+ * Run a docker command without a command file
+ */
+int exec_docker_command(char *docker_command, char **argv,
+    int argc, int optind);
+
 /*
  * Compile the regex_str and determine if the input string matches.
  * Return 0 on match, 1 of non-match.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
index b69546a..c54fd3e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
@@ -36,7 +36,7 @@ static void display_usage(FILE *stream) {
   fprintf(stream,
     "Usage: container-executor --checksetup\n"
     "       container-executor --mount-cgroups <hierarchy> "
-    "<controller=path>...\n" );
+    "<controller=path>\n" );
 
   if(is_tc_support_enabled()) {
     fprintf(stream,
@@ -52,10 +52,15 @@ static void display_usage(FILE *stream) {
 
   if(is_docker_support_enabled()) {
     fprintf(stream,
-      "       container-executor --run-docker <command-file>\n");
+      "       container-executor --run-docker <command-file>\n"
+      "       container-executor --remove-docker-container <container_id>\n"
+      "       container-executor --inspect-docker-container <container_id>\n");
   } else {
     fprintf(stream,
-      "[DISABLED] container-executor --run-docker <command-file>\n");
+      "[DISABLED] container-executor --run-docker <command-file>\n"
+      "[DISABLED] container-executor --remove-docker-container <container_id>\n"
+      "[DISABLED] container-executor --inspect-docker-container "
+      "<format> ... <container_id>\n");
   }
 
   fprintf(stream,
@@ -331,6 +336,36 @@ static int validate_arguments(int argc, char **argv , int *operation) {
     }
   }
 
+  if (strcmp("--remove-docker-container", argv[1]) == 0) {
+    if(is_docker_support_enabled()) {
+      if (argc != 3) {
+        display_usage(stdout);
+        return INVALID_ARGUMENT_NUMBER;
+      }
+      optind++;
+      *operation = REMOVE_DOCKER_CONTAINER;
+      return 0;
+    } else {
+        display_feature_disabled_message("docker");
+        return FEATURE_DISABLED;
+    }
+  }
+
+  if (strcmp("--inspect-docker-container", argv[1]) == 0) {
+    if(is_docker_support_enabled()) {
+      if (argc != 4) {
+        display_usage(stdout);
+        return INVALID_ARGUMENT_NUMBER;
+      }
+      optind++;
+      *operation = INSPECT_DOCKER_CONTAINER;
+      return 0;
+    } else {
+        display_feature_disabled_message("docker");
+        return FEATURE_DISABLED;
+    }
+  }
+
   /* Now we have to validate 'run as user' operations that don't use
     a 'long option' - we should fix this at some point. The validation/argument
     parsing here is extensive enough that it done in a separate function */
@@ -561,6 +596,12 @@ int main(int argc, char **argv) {
   case RUN_DOCKER:
     exit_code = run_docker(cmd_input.docker_command_file);
     break;
+  case REMOVE_DOCKER_CONTAINER:
+    exit_code = exec_docker_command("rm", argv, argc, optind);
+    break;
+  case INSPECT_DOCKER_CONTAINER:
+    exit_code = exec_docker_command("inspect", argv, argc, optind);
+    break;
   case RUN_AS_USER_INITIALIZE_CONTAINER:
     exit_code = set_user(cmd_input.run_as_user_name);
     if (exit_code != 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/883f6822/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java
index a230d4d..50d00bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerCommandExecutor.java
@@ -153,14 +153,14 @@ public class TestDockerCommandExecutor {
         env, configuration, mockExecutor, false, nmContext);
     List<PrivilegedOperation> ops = MockPrivilegedOperationCaptor
         .capturePrivilegedOperations(mockExecutor, 1, true);
-    List<String> dockerCommands = getValidatedDockerCommands(ops);
+    PrivilegedOperation privOp = ops.get(0);
+    List<String> args = privOp.getArguments();
     assertEquals(1, ops.size());
-    assertEquals(PrivilegedOperation.OperationType.RUN_DOCKER_CMD.name(),
-        ops.get(0).getOperationType().name());
-    assertEquals(3, dockerCommands.size());
-    assertEquals("[docker-command-execution]", dockerCommands.get(0));
-    assertEquals("  docker-command=rm", dockerCommands.get(1));
-    assertEquals("  name=" + MOCK_CONTAINER_ID, dockerCommands.get(2));
+    assertEquals(PrivilegedOperation.OperationType.
+        REMOVE_DOCKER_CONTAINER.name(),
+        privOp.getOperationType().name());
+    assertEquals(1, args.size());
+    assertEquals(MOCK_CONTAINER_ID, args.get(0));
   }
 
   @Test
@@ -188,16 +188,15 @@ public class TestDockerCommandExecutor {
         env, configuration, mockExecutor, false, nmContext);
     List<PrivilegedOperation> ops = MockPrivilegedOperationCaptor
         .capturePrivilegedOperations(mockExecutor, 1, true);
-    List<String> dockerCommands = getValidatedDockerCommands(ops);
+    PrivilegedOperation privOp = ops.get(0);
+    List<String> args = privOp.getArguments();
     assertEquals(1, ops.size());
-    assertEquals(PrivilegedOperation.OperationType.RUN_DOCKER_CMD.name(),
-        ops.get(0).getOperationType().name());
-    assertEquals(4, dockerCommands.size());
-    assertEquals("[docker-command-execution]", dockerCommands.get(0));
-    assertEquals("  docker-command=inspect", dockerCommands.get(1));
-    assertEquals("  format={{.State.Status}}", dockerCommands.get(2));
-    assertEquals("  name=" + MOCK_CONTAINER_ID, dockerCommands.get(3));
-
+    assertEquals(PrivilegedOperation.OperationType.
+        INSPECT_DOCKER_CONTAINER.name(),
+        privOp.getOperationType().name());
+    assertEquals(2, args.size());
+    assertEquals("--format={{.State.Status}}", args.get(0));
+    assertEquals(MOCK_CONTAINER_ID, args.get(1));
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/35] hadoop git commit: MAPREDUCE-7073. Optimize TokenCache#obtainTokensForNamenodesInternal

Posted by xy...@apache.org.
MAPREDUCE-7073. Optimize TokenCache#obtainTokensForNamenodesInternal

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1a95a452
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1a95a452
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1a95a452

Branch: refs/heads/HDDS-4
Commit: 1a95a4524a8c6c7be601ce8b92640a6a76164a2c
Parents: 3726926
Author: Bibin A Chundatt <bi...@apache.org>
Authored: Wed May 2 16:14:28 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed May 2 16:14:28 2018 +0900

----------------------------------------------------------------------
 .../hadoop/mapreduce/security/TokenCache.java     | 14 +++++++++-----
 .../hadoop/mapreduce/security/TestTokenCache.java | 18 +++++++++---------
 2 files changed, 18 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a95a452/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
index 12fced9..1156c67 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -96,8 +97,9 @@ public class TokenCache {
     for(Path p: ps) {
       fsSet.add(p.getFileSystem(conf));
     }
+    String masterPrincipal = Master.getMasterPrincipal(conf);
     for (FileSystem fs : fsSet) {
-      obtainTokensForNamenodesInternal(fs, credentials, conf);
+      obtainTokensForNamenodesInternal(fs, credentials, conf, masterPrincipal);
     }
   }
 
@@ -122,15 +124,17 @@ public class TokenCache {
    * @param conf
    * @throws IOException
    */
-  static void obtainTokensForNamenodesInternal(FileSystem fs, 
-      Credentials credentials, Configuration conf) throws IOException {
+  static void obtainTokensForNamenodesInternal(FileSystem fs,
+      Credentials credentials, Configuration conf, String renewer)
+      throws IOException {
     // RM skips renewing token with empty renewer
     String delegTokenRenewer = "";
     if (!isTokenRenewalExcluded(fs, conf)) {
-      delegTokenRenewer = Master.getMasterPrincipal(conf);
-      if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+      if (StringUtils.isEmpty(renewer)) {
         throw new IOException(
             "Can't get Master Kerberos principal for use as renewer");
+      } else {
+        delegTokenRenewer = renewer;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a95a452/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java
index 127f8ae..a44e533 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java
@@ -56,8 +56,8 @@ public class TestTokenCache {
   @Test
   public void testObtainTokens() throws Exception {
     Credentials credentials = new Credentials();
-    FileSystem fs = mock(FileSystem.class);  
-    TokenCache.obtainTokensForNamenodesInternal(fs, credentials, conf);
+    FileSystem fs = mock(FileSystem.class);
+    TokenCache.obtainTokensForNamenodesInternal(fs, credentials, conf, renewer);
     verify(fs).addDelegationTokens(eq(renewer), eq(credentials));
   }
 
@@ -105,23 +105,23 @@ public class TestTokenCache {
     checkToken(creds, newerToken1);
     
     // get token for fs1, see that fs2's token was loaded 
-    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf, renewer);
     checkToken(creds, newerToken1, token2);
     
     // get token for fs2, nothing should change since already present
-    TokenCache.obtainTokensForNamenodesInternal(fs2, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs2, creds, conf, renewer);
     checkToken(creds, newerToken1, token2);
     
     // get token for fs3, should only add token for fs3
-    TokenCache.obtainTokensForNamenodesInternal(fs3, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs3, creds, conf, renewer);
     Token<?> token3 = creds.getToken(new Text(fs3.getCanonicalServiceName()));
     assertTrue(token3 != null);
     checkToken(creds, newerToken1, token2, token3);
     
     // be paranoid, check one last time that nothing changes
-    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf);
-    TokenCache.obtainTokensForNamenodesInternal(fs2, creds, conf);
-    TokenCache.obtainTokensForNamenodesInternal(fs3, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf, renewer);
+    TokenCache.obtainTokensForNamenodesInternal(fs2, creds, conf, renewer);
+    TokenCache.obtainTokensForNamenodesInternal(fs3, creds, conf, renewer);
     checkToken(creds, newerToken1, token2, token3);
   }
 
@@ -202,7 +202,7 @@ public class TestTokenCache {
     // wait to set, else the obtain tokens call above will fail with FNF
     conf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, binaryTokenFile);
     creds.writeTokenStorageFile(new Path(binaryTokenFile), conf);
-    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf, renewer);
     String fs_addr = fs1.getCanonicalServiceName();
     Token<?> nnt = TokenCache.getDelegationToken(creds, fs_addr);
     assertNotNull("Token for nn is null", nnt);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/35] hadoop git commit: HDFS-11807. libhdfs++: Get minidfscluster tests running under valgrind. Contributed by Anatoli Shein.

Posted by xy...@apache.org.
HDFS-11807. libhdfs++: Get minidfscluster tests running under valgrind.  Contributed by Anatoli Shein.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/19ae588f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/19ae588f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/19ae588f

Branch: refs/heads/HDDS-4
Commit: 19ae588fde9930c042cdb2848b8a1a0ff514b575
Parents: fe649bb
Author: James Clampffer <jh...@apache.org>
Authored: Wed May 2 11:49:12 2018 -0400
Committer: James Clampffer <jh...@apache.org>
Committed: Wed May 2 11:49:12 2018 -0400

----------------------------------------------------------------------
 .../src/main/native/libhdfs-tests/expect.h      |  60 +++++
 .../libhdfs-tests/test_libhdfs_mini_stress.c    | 253 ++++++++++++++-----
 .../src/main/native/libhdfspp/CMakeLists.txt    |   2 +-
 .../main/native/libhdfspp/tests/CMakeLists.txt  |   6 +
 .../main/native/libhdfspp/tests/memcheck.supp   |  27 ++
 5 files changed, 279 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h
index 528c96f..d843b67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h
@@ -132,6 +132,54 @@ struct hdfsFile_internal;
         } \
     } while (0);
 
+#define EXPECT_INT_LT(x, y) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ >= (y)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "code %d (errno: %d): expected less than %d\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (y)); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_INT_LE(x, y) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ > (y)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "code %d (errno: %d): expected less than or equal %d\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (y)); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_INT_GT(x, y) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ <= (y)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "code %d (errno: %d): expected greater than %d\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (y)); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_INT_GE(x, y) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ < (y)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "code %d (errno: %d): expected greater than or equal %d\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (y)); \
+            return -1; \
+        } \
+    } while (0);
+
 #define EXPECT_INT64_EQ(x, y) \
     do { \
         int64_t __my_ret__ = y; \
@@ -144,6 +192,18 @@ struct hdfsFile_internal;
         } \
     } while (0);
 
+#define ASSERT_INT64_EQ(x, y) \
+    do { \
+        int64_t __my_ret__ = y; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ != (x)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "value %"PRId64" (errno: %d): expected %"PRId64"\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (x)); \
+            exit(EXIT_FAILURE); \
+        } \
+    } while (0);
+
 #define EXPECT_UINT64_EQ(x, y) \
     do { \
         uint64_t __my_ret__ = y; \

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
index dca4782..9054287 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+#include "common/util_c.h"
 #include "expect.h"
 #include "hdfs/hdfs.h"
 #include "hdfspp/hdfs_ext.h"
@@ -24,10 +25,15 @@
 
 #include <errno.h>
 #include <inttypes.h>
+#include <pwd.h>
 #include <stdint.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/wait.h>
+#include <unistd.h>
 
 #define TO_STR_HELPER(X) #X
 #define TO_STR(X) TO_STR_HELPER(X)
@@ -44,6 +50,8 @@
 
 #define TLH_DEFAULT_IPC_CLIENT_CONNECT_RETRY_INTERVAL_MS 5
 
+#define MAX_DIGITS_IN_INT 10
+
 #ifndef RANDOM_ERROR_RATIO
 #define RANDOM_ERROR_RATIO 1000000000
 #endif
@@ -62,15 +70,13 @@ struct tlhThreadInfo {
 
 };
 
-static int hdfsNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs,
+static int hdfsNameNodeConnect(tPort port, hdfsFS *fs,
                                const char *username)
 {
   int ret;
-  tPort port;
   hdfsFS hdfs;
   struct hdfsBuilder *bld;
 
-  port = (tPort)nmdGetNameNodePort(cl);
   if (port < 0) {
     fprintf(stderr, "hdfsNameNodeConnect: nmdGetNameNodePort "
             "returned error %d\n", port);
@@ -104,6 +110,44 @@ static int hdfsNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs,
   return 0;
 }
 
+#ifdef VALGRIND
+static int hdfsCurlData(const char *host, const tPort port, const char *dirNm,
+                         const char *fileNm, tSize fileSz)
+{
+  int ret;
+  const char *content;
+  content = fileNm;
+  char tmpFile[14] = "stress_XXXXXX";
+
+  // Retrieve user id (always successful)
+  uid_t uid = geteuid();
+  // Retrieve password struct entry
+  struct passwd *pw;
+  EXPECT_NONNULL(pw = getpwuid(uid));
+
+  int fd = -1;
+  EXPECT_NONNEGATIVE(fd = mkstemp(tmpFile));
+
+  tSize sz = 0;
+  while (sz < fileSz) {
+    EXPECT_NONNEGATIVE(ret = write(fd, content, strlen(content)));
+    sz += ret;
+  }
+
+  int curlSize = 200;
+  char curlStr[curlSize];
+  ret = snprintf(curlStr,curlSize,"curl -L -i -X PUT -T %s \"http://%s:%d/webhdfs/v1%s?op=CREATE&overwrite=true&user.name=%s\"",tmpFile,host,(int)port,fileNm,pw->pw_name);
+  //Check for errors during snprintf
+  EXPECT_NONNEGATIVE(ret);
+  //Check for truncation during snprintf
+  EXPECT_INT_LT(ret, curlSize);
+
+  EXPECT_ZERO(system(curlStr));
+  EXPECT_ZERO(unlink(tmpFile));
+
+  return 0;
+}
+#else
 static int hdfsWriteData(hdfsFS hdfs, const char *dirNm,
                          const char *fileNm, tSize fileSz)
 {
@@ -142,6 +186,7 @@ static int hdfsWriteData(hdfsFS hdfs, const char *dirNm,
   EXPECT_ZERO(hdfsCloseFile(hdfs, file));
   return 0;
 }
+#endif
 
 static int fileEventCallback1(const char * event, const char * cluster, const char * file, int64_t value, int64_t cookie)
 {
@@ -223,6 +268,7 @@ static int doTestHdfsMiniStress(struct tlhThreadInfo *ti, int randomErr)
   fprintf(stderr, "testHdfsMiniStress(threadIdx=%d): finished read loop\n",
           ti->threadIdx);
   EXPECT_ZERO(nErrs);
+  hdfsFreeFileInfo(fileInfo, 1);
   return 0;
 }
 
@@ -275,76 +321,147 @@ static int checkFailures(struct tlhThreadInfo *ti, int tlhNumThreads)
 
 /**
  * Test intended to stress libhdfs client with concurrent requests. Currently focused
- * on concurrent reads.
+ * on concurrent reads. In order to run this test under valgrind and avoid JVM issues
+ * we fork a child process that runs a mini dfs cluster, and the parent process
+ * communicates with it using a socket pair.
  */
-int main(void)
+int main(int argc, char *argv[])
 {
-  int i, tlhNumThreads;
-  char *dirNm, *fileNm;
-  tSize fileSz;
-  const char *tlhNumThreadsStr, *tlhNumDNsStr;
-  hdfsFS hdfs = NULL;
-  struct NativeMiniDfsCluster* tlhCluster;
-  struct tlhThreadInfo ti[TLH_MAX_THREADS];
-  struct NativeMiniDfsConf conf = {
+  tPort port;
+#ifdef VALGRIND
+  int httpPort;
+  char * httpHost;
+  size_t hostSize;
+  int fds[2];
+  static const int parentsocket = 0;
+  static const int childsocket = 1;
+  int status;
+  // If there is an argument, the child code executes and starts a mini dfs cluster
+  if (argc > 1) {
+    // The argument contains child socket
+    fds[childsocket] = (int) strtol(argv[1],NULL,10);
+#endif
+    const char *tlhNumDNsStr;
+    struct NativeMiniDfsCluster* tlhCluster;
+    struct NativeMiniDfsConf conf = {
       1, /* doFormat */
-  };
-
-  dirNm = "/tlhMiniStressData";
-  fileNm = "/tlhMiniStressData/file";
-  fileSz = 2*1024*1024;
-
-  tlhNumDNsStr = getenv("TLH_NUM_DNS");
-  if (!tlhNumDNsStr) {
-    tlhNumDNsStr = "1";
-  }
-  conf.numDataNodes = atoi(tlhNumDNsStr);
-  if ((conf.numDataNodes <= 0) || (conf.numDataNodes > TLH_MAX_DNS)) {
-    fprintf(stderr, "testLibHdfsMiniStress: must have a number of datanodes "
-            "between 1 and %d inclusive, not %d\n",
-            TLH_MAX_DNS, conf.numDataNodes);
-    return EXIT_FAILURE;
-  }
-
-  tlhNumThreadsStr = getenv("TLH_NUM_THREADS");
-  if (!tlhNumThreadsStr) {
-    tlhNumThreadsStr = "8";
-  }
-  tlhNumThreads = atoi(tlhNumThreadsStr);
-  if ((tlhNumThreads <= 0) || (tlhNumThreads > TLH_MAX_THREADS)) {
-    fprintf(stderr, "testLibHdfsMiniStress: must have a number of threads "
-            "between 1 and %d inclusive, not %d\n",
-            TLH_MAX_THREADS, tlhNumThreads);
-    return EXIT_FAILURE;
-  }
-  memset(&ti[0], 0, sizeof(ti));
-  for (i = 0; i < tlhNumThreads; i++) {
-    ti[i].threadIdx = i;
-  }
-
-  tlhCluster = nmdCreate(&conf);
-  EXPECT_NONNULL(tlhCluster);
-  EXPECT_ZERO(nmdWaitClusterUp(tlhCluster));
+      1, /* webhdfs */
+      0, /* webhdfs port */
+      1  /* shortcircuit */
+    };
+    tlhNumDNsStr = getenv("TLH_NUM_DNS");
+    if (!tlhNumDNsStr) {
+      tlhNumDNsStr = "1";
+    }
+    conf.numDataNodes = atoi(tlhNumDNsStr);
+    if ((conf.numDataNodes <= 0) || (conf.numDataNodes > TLH_MAX_DNS)) {
+      fprintf(stderr, "testLibHdfsMiniStress: must have a number of datanodes "
+              "between 1 and %d inclusive, not %d\n",
+              TLH_MAX_DNS, conf.numDataNodes);
+      return EXIT_FAILURE;
+    }
+    tlhCluster = nmdCreate(&conf);
+    EXPECT_NONNULL(tlhCluster);
+    EXPECT_ZERO(nmdWaitClusterUp(tlhCluster));
+    port = (tPort)nmdGetNameNodePort(tlhCluster);
+#ifdef VALGRIND
+    EXPECT_ZERO(nmdGetNameNodeHttpAddress(tlhCluster, &httpPort, (const char **) &httpHost));
+    hostSize = strlen(httpHost) + 1;
+    // The child is sending hdfs port, webhdfs port, hostname size, and hostname to the parent
+    ASSERT_INT64_EQ(write(fds[childsocket], &port, sizeof(tPort)), sizeof(tPort));
+    ASSERT_INT64_EQ(write(fds[childsocket], &httpPort, sizeof(int)), sizeof(int));
+    ASSERT_INT64_EQ(write(fds[childsocket], &hostSize, sizeof(size_t)), sizeof(size_t));
+    ASSERT_INT64_EQ(write(fds[childsocket], httpHost, hostSize), hostSize);
+    free(httpHost);
+    // The child is waiting for the parent to finish and send a message
+    ASSERT_INT64_EQ(read(fds[childsocket], &port, sizeof(tPort)), sizeof(tPort));
+    EXPECT_ZERO(nmdShutdown(tlhCluster));
+    nmdFree(tlhCluster);
+  } else { // If there is no argument, the parent code executes
+#endif
+    hdfsFS hdfs = NULL;
+    int i, tlhNumThreads;
+    char *dirNm, *fileNm;
+    tSize fileSz;
+    const char *tlhNumThreadsStr;
+    struct tlhThreadInfo ti[TLH_MAX_THREADS];
+
+    dirNm = "/tlhMiniStressData";
+    fileNm = "/tlhMiniStressData/file";
+    fileSz = 2*1024*1024;
+
+    tlhNumThreadsStr = getenv("TLH_NUM_THREADS");
+    if (!tlhNumThreadsStr) {
+      tlhNumThreadsStr = "8";
+    }
+    tlhNumThreads = atoi(tlhNumThreadsStr);
+    if ((tlhNumThreads <= 0) || (tlhNumThreads > TLH_MAX_THREADS)) {
+      fprintf(stderr, "testLibHdfsMiniStress: must have a number of threads "
+              "between 1 and %d inclusive, not %d\n",
+              TLH_MAX_THREADS, tlhNumThreads);
+      return EXIT_FAILURE;
+    }
+    memset(&ti[0], 0, sizeof(ti));
+    for (i = 0; i < tlhNumThreads; i++) {
+      ti[i].threadIdx = i;
+    }
 
-  EXPECT_ZERO(hdfsNameNodeConnect(tlhCluster, &hdfs, NULL));
+#ifdef VALGRIND
+    EXPECT_ZERO(socketpair(PF_LOCAL, SOCK_STREAM, 0, fds));
+    // Forking off a child to execute JVM stuff
+    pid_t pid = fork();
+    if(pid == 0){
+      // The child execs this program from the beginning and passes
+      // its file descriptor as a command line argument.
+      char ch_fd[MAX_DIGITS_IN_INT + 1];
+      sprintf(ch_fd, "%d", fds[childsocket]);
+      // This has to be done with exec() to prevent valgrind from tracing the child
+      execl(argv[0], argv[0], ch_fd, NULL);
+      // This code should never execute
+      fprintf(stderr, "execl() failed.\n");
+      return EXIT_FAILURE;
+    }
+    close(fds[childsocket]);
+    // The parent is receiving hdfs port, webhdfs port, hostname size, and hostname from the child
+    ASSERT_INT64_EQ(read(fds[parentsocket], &port, sizeof(tPort)), sizeof(tPort));
+    ASSERT_INT64_EQ(read(fds[parentsocket], &httpPort, sizeof(int)), sizeof(int));
+    ASSERT_INT64_EQ(read(fds[parentsocket], &hostSize, sizeof(size_t)), sizeof(size_t));
+    httpHost = malloc(hostSize);
+    ASSERT_INT64_EQ(read(fds[parentsocket], httpHost, hostSize), hostSize);
+    EXPECT_ZERO(hdfsNameNodeConnect(port, &hdfs, NULL));
+    EXPECT_ZERO(hdfsCurlData(httpHost, httpPort, dirNm, fileNm, fileSz));
+    free(httpHost);
+#else
+    EXPECT_ZERO(hdfsNameNodeConnect(port, &hdfs, NULL));
+    EXPECT_ZERO(hdfsWriteData(hdfs, dirNm, fileNm, fileSz));
+#endif
 
-  // Single threaded writes for now.
-  EXPECT_ZERO(hdfsWriteData(hdfs, dirNm, fileNm, fileSz));
+    // Multi-threaded reads.
+    for (i = 0; i < tlhNumThreads; i++) {
+      ti[i].theThread.start = testHdfsMiniStress;
+      ti[i].theThread.arg = &ti[i];
+      ti[i].hdfs = hdfs;
+      ti[i].fileNm = fileNm;
+      EXPECT_ZERO(threadCreate(&ti[i].theThread));
+    }
+    for (i = 0; i < tlhNumThreads; i++) {
+      EXPECT_ZERO(threadJoin(&ti[i].theThread));
+    }
 
-  // Multi-threaded reads.
-  for (i = 0; i < tlhNumThreads; i++) {
-    ti[i].theThread.start = testHdfsMiniStress;
-    ti[i].theThread.arg = &ti[i];
-    ti[i].hdfs = hdfs;
-    ti[i].fileNm = fileNm;
-    EXPECT_ZERO(threadCreate(&ti[i].theThread));
-  }
-  for (i = 0; i < tlhNumThreads; i++) {
-    EXPECT_ZERO(threadJoin(&ti[i].theThread));
+    EXPECT_ZERO(hdfsDisconnect(hdfs));
+    EXPECT_ZERO(checkFailures(ti, tlhNumThreads));
+#ifdef VALGRIND
+    //Send this message to the child to notify it that it can now shut down
+    ASSERT_INT64_EQ(write(fds[parentsocket], &port, sizeof(tPort)), sizeof(tPort));
+    // Wait for the child to exit and verify it returned EXIT_SUCCESS
+    waitpid(pid, &status, 0);
+    EXPECT_ZERO(status);
   }
-
-  EXPECT_ZERO(hdfsDisconnect(hdfs));
-  EXPECT_ZERO(nmdShutdown(tlhCluster));
-  nmdFree(tlhCluster);
-  return checkFailures(ti, tlhNumThreads);
+#else
+    EXPECT_ZERO(nmdShutdown(tlhCluster));
+    nmdFree(tlhCluster);
+#endif
+  // Clean up static data and prevent valgrind memory leaks
+  ShutdownProtobufLibrary_C();
+  return EXIT_SUCCESS;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
index 94b1b56..63fa80d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
@@ -90,7 +90,7 @@ if (NOT PROTOC_IS_COMPATIBLE)
 endif (NOT PROTOC_IS_COMPATIBLE)
 
 find_program(MEMORYCHECK_COMMAND valgrind HINTS ${VALGRIND_DIR} )
-set(MEMORYCHECK_COMMAND_OPTIONS "--trace-children=yes --leak-check=full --error-exitcode=1")
+set(MEMORYCHECK_COMMAND_OPTIONS "--trace-children=no --leak-check=full --error-exitcode=1 --suppressions=${PROJECT_SOURCE_DIR}/tests/memcheck.supp")
 message(STATUS "valgrind location: ${MEMORYCHECK_COMMAND}")
 
 if (REQUIRE_VALGRIND AND MEMORYCHECK_COMMAND MATCHES "MEMORYCHECK_COMMAND-NOTFOUND" )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
index 3331935..6157902 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
@@ -141,6 +141,7 @@ include_directories (
 )
 
 add_library(hdfspp_test_shim_static STATIC hdfs_shim.c libhdfs_wrapper.c libhdfspp_wrapper.cc ${LIBHDFSPP_BINDING_C}/hdfs.cc)
+add_library(hdfspp_test_static STATIC ${LIBHDFSPP_BINDING_C}/hdfs.cc)
 
 # TODO: get all of the mini dfs library bits here in one place
 # add_library(hdfspp_mini_cluster     native_mini_dfs ${JAVA_JVM_LIBRARY} )
@@ -155,6 +156,11 @@ build_libhdfs_test(hdfspp_mini_dfs_smoke hdfspp_test_shim_static ${CMAKE_CURRENT
 link_libhdfs_test (hdfspp_mini_dfs_smoke hdfspp_test_shim_static fs reader rpc proto common connection gmock_main ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} native_mini_dfs ${JAVA_JVM_LIBRARY} ${SASL_LIBRARIES})
 add_libhdfs_test  (hdfspp_mini_dfs_smoke hdfspp_test_shim_static)
 
+build_libhdfs_test(libhdfs_mini_stress_valgrind hdfspp_test_static expect.c test_libhdfs_mini_stress.c ${OS_DIR}/thread.c)
+link_libhdfs_test(libhdfs_mini_stress_valgrind hdfspp_test_static fs reader rpc proto common connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} native_mini_dfs ${JAVA_JVM_LIBRARY} ${SASL_LIBRARIES})
+add_memcheck_test(libhdfs_mini_stress_valgrind_hdfspp_test_static libhdfs_mini_stress_valgrind_hdfspp_test_static)
+set_target_properties(libhdfs_mini_stress_valgrind_hdfspp_test_static PROPERTIES COMPILE_DEFINITIONS "VALGRIND")
+
 build_libhdfs_test(libhdfs_mini_stress hdfspp_test_shim_static expect.c test_libhdfs_mini_stress.c ${OS_DIR}/thread.c)
 link_libhdfs_test(libhdfs_mini_stress hdfspp_test_shim_static fs reader rpc proto common connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} native_mini_dfs ${JAVA_JVM_LIBRARY} ${SASL_LIBRARIES})
 add_libhdfs_test(libhdfs_mini_stress hdfspp_test_shim_static)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/memcheck.supp
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/memcheck.supp b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/memcheck.supp
new file mode 100644
index 0000000..cf80d07
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/memcheck.supp
@@ -0,0 +1,27 @@
+#
+# 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.
+
+{
+   noai6ai_cached suppression
+   Memcheck:Free
+   fun:free
+   fun:__libc_freeres
+   fun:_vgnU_freeres
+   fun:__run_exit_handlers
+   fun:exit
+   ...
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/35] hadoop git commit: HADOOP-15377. Improve debug messages in MetricsConfig.java

Posted by xy...@apache.org.
HADOOP-15377. Improve debug messages in MetricsConfig.java

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/33768724
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/33768724
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/33768724

Branch: refs/heads/HDDS-4
Commit: 33768724ff99d4966c24c9553eef207ed31a76d3
Parents: 1a95a45
Author: BELUGA BEHR <da...@gmail.com>
Authored: Wed May 2 17:09:22 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed May 2 17:09:22 2018 +0900

----------------------------------------------------------------------
 .../hadoop/metrics2/impl/MetricsConfig.java     | 50 ++++++++++++--------
 1 file changed, 30 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/33768724/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java
index ac4a24e..027450c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsConfig.java
@@ -118,20 +118,23 @@ class MetricsConfig extends SubsetConfiguration {
                 .setListDelimiterHandler(new DefaultListDelimiterHandler(',')))
               .getConfiguration()
               .interpolatedConfiguration();
-        LOG.info("loaded properties from "+ fname);
-        LOG.debug(toString(cf));
+        LOG.info("Loaded properties from {}", fname);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Properties: {}", toString(cf));
+        }
         MetricsConfig mc = new MetricsConfig(cf, prefix);
-        LOG.debug(mc.toString());
+        LOG.debug("Metrics Config: {}", mc);
         return mc;
       } catch (ConfigurationException e) {
         // Commons Configuration defines the message text when file not found
         if (e.getMessage().startsWith("Could not locate")) {
+          LOG.debug("Could not locate file {}", fname, e);
           continue;
         }
         throw new MetricsConfigException(e);
       }
     }
-    LOG.warn("Cannot locate configuration: tried "+
+    LOG.warn("Cannot locate configuration: tried " +
              Joiner.on(",").join(fileNames));
     // default to an empty configuration
     return new MetricsConfig(new PropertiesConfiguration(), prefix);
@@ -168,7 +171,6 @@ class MetricsConfig extends SubsetConfiguration {
 
   Iterable<String> keys() {
     return new Iterable<String>() {
-      @SuppressWarnings("unchecked")
       @Override
       public Iterator<String> iterator() {
         return (Iterator<String>) getKeys();
@@ -186,21 +188,21 @@ class MetricsConfig extends SubsetConfiguration {
     Object value = super.getPropertyInternal(key);
     if (value == null) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("poking parent '"+ getParent().getClass().getSimpleName() +
-                  "' for key: "+ key);
+        LOG.debug("poking parent '" + getParent().getClass().getSimpleName() +
+                  "' for key: " + key);
       }
       return getParent().getProperty(key.startsWith(PREFIX_DEFAULT) ? key
                                      : PREFIX_DEFAULT + key);
     }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("returning '"+ value +"' for key: "+ key);
-    }
+    LOG.debug("Returning '{}' for key: {}", value, key);
     return value;
   }
 
   <T extends MetricsPlugin> T getPlugin(String name) {
     String clsName = getClassName(name);
-    if (clsName == null) return null;
+    if (clsName == null) {
+      return null;
+    }
     try {
       Class<?> cls = Class.forName(clsName, true, getPluginLoader());
       @SuppressWarnings("unchecked")
@@ -213,9 +215,9 @@ class MetricsConfig extends SubsetConfiguration {
   }
 
   String getClassName(String prefix) {
-    String classKey = prefix.isEmpty() ? "class" : prefix +".class";
+    String classKey = prefix.isEmpty() ? "class" : prefix.concat(".class");
     String clsName = getString(classKey);
-    LOG.debug(clsName);
+    LOG.debug("Class name for prefix {} is {}", prefix, clsName);
     if (clsName == null || clsName.isEmpty()) {
       return null;
     }
@@ -223,25 +225,29 @@ class MetricsConfig extends SubsetConfiguration {
   }
 
   ClassLoader getPluginLoader() {
-    if (pluginLoader != null) return pluginLoader;
+    if (pluginLoader != null) {
+      return pluginLoader;
+    }
     final ClassLoader defaultLoader = getClass().getClassLoader();
     Object purls = super.getProperty(PLUGIN_URLS_KEY);
-    if (purls == null) return defaultLoader;
+    if (purls == null) {
+      return defaultLoader;
+    }
     Iterable<String> jars = SPLITTER.split((String) purls);
     int len = Iterables.size(jars);
-    if ( len > 0) {
+    if (len > 0) {
       final URL[] urls = new URL[len];
       try {
         int i = 0;
         for (String jar : jars) {
-          LOG.debug(jar);
+          LOG.debug("Parsing URL for {}", jar);
           urls[i++] = new URL(jar);
         }
       } catch (Exception e) {
         throw new MetricsConfigException(e);
       }
       if (LOG.isDebugEnabled()) {
-        LOG.debug("using plugin jars: "+ Iterables.toString(jars));
+        LOG.debug("Using plugin jars: {}", Iterables.toString(jars));
       }
       pluginLoader = doPrivileged(new PrivilegedAction<ClassLoader>() {
         @Override public ClassLoader run() {
@@ -259,9 +265,13 @@ class MetricsConfig extends SubsetConfiguration {
   MetricsFilter getFilter(String prefix) {
     // don't create filter instances without out options
     MetricsConfig conf = subset(prefix);
-    if (conf.isEmpty()) return null;
+    if (conf.isEmpty()) {
+      return null;
+    }
     MetricsFilter filter = getPlugin(prefix);
-    if (filter != null) return filter;
+    if (filter != null) {
+      return filter;
+    }
     // glob filter is assumed if pattern is specified but class is not.
     filter = new GlobFilter();
     filter.init(conf);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/35] hadoop git commit: YARN-8222. Fix potential NPE when gets RMApp from RM context. Contributed by Tao Yang.

Posted by xy...@apache.org.
YARN-8222. Fix potential NPE when gets RMApp from RM context. Contributed by Tao Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/251f5288
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/251f5288
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/251f5288

Branch: refs/heads/HDDS-4
Commit: 251f528814c4a4647cac0af6effb9a73135db180
Parents: 3265b55
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed May 2 17:54:46 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Wed May 2 17:54:46 2018 +0800

----------------------------------------------------------------------
 .../rmcontainer/RMContainerImpl.java            | 30 +++++++++++---------
 .../scheduler/SchedulerApplicationAttempt.java  | 13 +++++----
 2 files changed, 23 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/251f5288/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 541621b..b5c8e7c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
@@ -737,21 +738,22 @@ public class RMContainerImpl implements RMContainer {
 
     private static void updateAttemptMetrics(RMContainerImpl container) {
       Resource resource = container.getContainer().getResource();
-      RMAppAttempt rmAttempt = container.rmContext.getRMApps()
-          .get(container.getApplicationAttemptId().getApplicationId())
-          .getCurrentAppAttempt();
-
-      if (rmAttempt != null) {
-        long usedMillis = container.finishTime - container.creationTime;
-        rmAttempt.getRMAppAttemptMetrics()
-            .updateAggregateAppResourceUsage(resource, usedMillis);
-        // If this is a preempted container, update preemption metrics
-        if (ContainerExitStatus.PREEMPTED == container.finishedStatus
-            .getExitStatus()) {
+      RMApp app = container.rmContext.getRMApps()
+          .get(container.getApplicationAttemptId().getApplicationId());
+      if (app != null) {
+        RMAppAttempt rmAttempt = app.getCurrentAppAttempt();
+        if (rmAttempt != null) {
+          long usedMillis = container.finishTime - container.creationTime;
           rmAttempt.getRMAppAttemptMetrics()
-              .updatePreemptionInfo(resource, container);
-          rmAttempt.getRMAppAttemptMetrics()
-              .updateAggregatePreemptedAppResourceUsage(resource, usedMillis);
+              .updateAggregateAppResourceUsage(resource, usedMillis);
+          // If this is a preempted container, update preemption metrics
+          if (ContainerExitStatus.PREEMPTED == container.finishedStatus
+              .getExitStatus()) {
+            rmAttempt.getRMAppAttemptMetrics()
+                .updatePreemptionInfo(resource, container);
+            rmAttempt.getRMAppAttemptMetrics()
+                .updateAggregatePreemptedAppResourceUsage(resource, usedMillis);
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/251f5288/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 857e736..005569c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -1241,12 +1241,13 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       return;
     }
 
-    RMAppAttempt attempt =
-        rmContext.getRMApps().get(attemptId.getApplicationId())
-          .getCurrentAppAttempt();
-    if (attempt != null) {
-      attempt.getRMAppAttemptMetrics().incNumAllocatedContainers(containerType,
-        requestType);
+    RMApp app = rmContext.getRMApps().get(attemptId.getApplicationId());
+    if (app != null) {
+      RMAppAttempt attempt = app.getCurrentAppAttempt();
+      if (attempt != null) {
+        attempt.getRMAppAttemptMetrics()
+            .incNumAllocatedContainers(containerType, requestType);
+      }
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/35] hadoop git commit: Revert "HADOOP-14445. Delegation tokens are not shared between KMS instances. Contributed by Xiao Chen and Rushabh S Shah."

Posted by xy...@apache.org.
Revert "HADOOP-14445. Delegation tokens are not shared between KMS instances. Contributed by Xiao Chen and Rushabh S Shah."

This reverts commit 583fa6ed48ad3df40bcaa9c591d5ccd07ce3ea81.

 Conflicts:
	hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSTokenRenewer.java
	hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a3a1552c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a3a1552c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a3a1552c

Branch: refs/heads/HDDS-4
Commit: a3a1552c33d5650fbd0a702369fccd21b8c9d3e2
Parents: 118bd75
Author: Xiao Chen <xi...@apache.org>
Authored: Fri May 4 20:29:20 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Mon May 7 13:32:27 2018 -0700

----------------------------------------------------------------------
 .../crypto/key/kms/KMSClientProvider.java       | 214 ++++----
 .../crypto/key/kms/KMSDelegationToken.java      |  22 +-
 .../crypto/key/kms/KMSLegacyTokenRenewer.java   |  56 --
 .../hadoop/crypto/key/kms/KMSTokenRenewer.java  | 103 ----
 .../hadoop/crypto/key/kms/package-info.java     |  18 -
 .../fs/CommonConfigurationKeysPublic.java       |  10 -
 .../web/DelegationTokenAuthenticatedURL.java    |  21 +-
 .../DelegationTokenAuthenticationHandler.java   |   8 +-
 .../web/DelegationTokenAuthenticator.java       |   2 +-
 .../java/org/apache/hadoop/util/KMSUtil.java    |  45 +-
 .../hadoop/util/KMSUtilFaultInjector.java       |  49 --
 ...apache.hadoop.security.token.TokenIdentifier |   1 -
 ...rg.apache.hadoop.security.token.TokenRenewer |   3 +-
 .../src/main/resources/core-default.xml         |  20 -
 .../crypto/key/kms/TestKMSClientProvider.java   | 162 ------
 .../kms/TestLoadBalancingKMSClientProvider.java |  67 +--
 .../org/apache/hadoop/util/TestKMSUtil.java     |  65 ---
 .../hadoop/crypto/key/kms/server/TestKMS.java   | 519 +++----------------
 18 files changed, 205 insertions(+), 1180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
index 45097ef..dddd358 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
@@ -36,9 +36,8 @@ import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenSelector;
+import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
 import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.hadoop.util.KMSUtil;
@@ -83,8 +82,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.KMS_CLIENT_COPY_LEGACY_TOKEN_KEY;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.KMS_CLIENT_COPY_LEGACY_TOKEN_DEFAULT;
 import static org.apache.hadoop.util.KMSUtil.checkNotEmpty;
 import static org.apache.hadoop.util.KMSUtil.checkNotNull;
 import static org.apache.hadoop.util.KMSUtil.parseJSONEncKeyVersion;
@@ -99,13 +96,16 @@ import static org.apache.hadoop.util.KMSUtil.parseJSONMetadata;
 public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     KeyProviderDelegationTokenExtension.DelegationTokenExtension {
 
-  public static final Logger LOG =
+  private static final Logger LOG =
       LoggerFactory.getLogger(KMSClientProvider.class);
 
   private static final String INVALID_SIGNATURE = "Invalid signature";
 
   private static final String ANONYMOUS_REQUESTS_DISALLOWED = "Anonymous requests are disallowed";
 
+  public static final String TOKEN_KIND_STR = KMSDelegationToken.TOKEN_KIND_STR;
+  public static final Text TOKEN_KIND = KMSDelegationToken.TOKEN_KIND;
+
   public static final String SCHEME_NAME = "kms";
 
   private static final String UTF8 = "UTF-8";
@@ -133,17 +133,12 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
   private static final ObjectWriter WRITER =
       new ObjectMapper().writerWithDefaultPrettyPrinter();
 
-  /* dtService defines the token service value for the kms token.
-   * The value can be legacy format which is ip:port format or it can be uri.
-   * If it's uri format, then the value is read from
-   * CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH at key
-   * provider creation time, and set to token's Service field.
-   * When a token is renewed / canceled, its Service field will be used to
-   * instantiate a KeyProvider, eliminating the need to read configs
-    * at that time.
-   */
   private final Text dtService;
-  private final boolean copyLegacyToken;
+
+  // Allow fallback to default kms server port 9600 for certain tests that do
+  // not specify the port explicitly in the kms provider url.
+  @VisibleForTesting
+  public static volatile boolean fallbackDefaultPortForTesting = false;
 
   private class EncryptedQueueRefiller implements
     ValueQueue.QueueRefiller<EncryptedKeyVersion> {
@@ -167,6 +162,68 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     }
   }
 
+  /**
+   * The KMS implementation of {@link TokenRenewer}.
+   */
+  public static class KMSTokenRenewer extends TokenRenewer {
+    private static final Logger LOG =
+        LoggerFactory.getLogger(KMSTokenRenewer.class);
+
+    @Override
+    public boolean handleKind(Text kind) {
+      return kind.equals(TOKEN_KIND);
+    }
+
+    @Override
+    public boolean isManaged(Token<?> token) throws IOException {
+      return true;
+    }
+
+    @Override
+    public long renew(Token<?> token, Configuration conf) throws IOException {
+      LOG.debug("Renewing delegation token {}", token);
+      KeyProvider keyProvider = KMSUtil.createKeyProvider(conf,
+          KeyProviderFactory.KEY_PROVIDER_PATH);
+      try {
+        if (!(keyProvider instanceof
+            KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
+          throw new IOException(String
+              .format("keyProvider %s cannot renew token [%s]",
+                  keyProvider == null ? "null" : keyProvider.getClass(),
+                  token));
+        }
+        return ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
+            keyProvider).renewDelegationToken(token);
+      } finally {
+        if (keyProvider != null) {
+          keyProvider.close();
+        }
+      }
+    }
+
+    @Override
+    public void cancel(Token<?> token, Configuration conf) throws IOException {
+      LOG.debug("Canceling delegation token {}", token);
+      KeyProvider keyProvider = KMSUtil.createKeyProvider(conf,
+          KeyProviderFactory.KEY_PROVIDER_PATH);
+      try {
+        if (!(keyProvider instanceof
+            KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
+          throw new IOException(String
+              .format("keyProvider %s cannot cancel token [%s]",
+                  keyProvider == null ? "null" : keyProvider.getClass(),
+                  token));
+        }
+        ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
+            keyProvider).cancelDelegationToken(token);
+      } finally {
+        if (keyProvider != null) {
+          keyProvider.close();
+        }
+      }
+    }
+  }
+
   public static class KMSEncryptedKeyVersion extends EncryptedKeyVersion {
     public KMSEncryptedKeyVersion(String keyName, String keyVersionName,
         byte[] iv, String encryptedVersionName, byte[] keyMaterial) {
@@ -226,13 +283,13 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           }
           hostsPart = t[0];
         }
-        return createProvider(conf, origUrl, port, hostsPart, providerUri);
+        return createProvider(conf, origUrl, port, hostsPart);
       }
       return null;
     }
 
-    private KeyProvider createProvider(Configuration conf, URL origUrl,
-        int port, String hostsPart, URI providerUri) throws IOException {
+    private KeyProvider createProvider(Configuration conf,
+        URL origUrl, int port, String hostsPart) throws IOException {
       String[] hosts = hostsPart.split(";");
       KMSClientProvider[] providers = new KMSClientProvider[hosts.length];
       for (int i = 0; i < hosts.length; i++) {
@@ -240,7 +297,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           providers[i] =
               new KMSClientProvider(
                   new URI("kms", origUrl.getProtocol(), hosts[i], port,
-                      origUrl.getPath(), null, null), conf, providerUri);
+                      origUrl.getPath(), null, null), conf);
         } catch (URISyntaxException e) {
           throw new IOException("Could not instantiate KMSProvider.", e);
         }
@@ -298,10 +355,17 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     }
   }
 
-  public KMSClientProvider(URI uri, Configuration conf, URI providerUri) throws
-      IOException {
+  public KMSClientProvider(URI uri, Configuration conf) throws IOException {
     super(conf);
     kmsUrl = createServiceURL(extractKMSPath(uri));
+    int kmsPort = kmsUrl.getPort();
+    if ((kmsPort == -1) && fallbackDefaultPortForTesting) {
+      kmsPort = 9600;
+    }
+
+    InetSocketAddress addr = new InetSocketAddress(kmsUrl.getHost(), kmsPort);
+    dtService = SecurityUtil.buildTokenService(addr);
+
     if ("https".equalsIgnoreCase(kmsUrl.getProtocol())) {
       sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
       try {
@@ -314,9 +378,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
             CommonConfigurationKeysPublic.KMS_CLIENT_TIMEOUT_SECONDS,
             CommonConfigurationKeysPublic.KMS_CLIENT_TIMEOUT_DEFAULT);
     authRetry = conf.getInt(AUTH_RETRY, DEFAULT_AUTH_RETRY);
-    copyLegacyToken = conf.getBoolean(KMS_CLIENT_COPY_LEGACY_TOKEN_KEY,
-        KMS_CLIENT_COPY_LEGACY_TOKEN_DEFAULT);
-
     configurator = new TimeoutConnConfigurator(timeout, sslFactory);
     encKeyVersionQueue =
         new ValueQueue<KeyProviderCryptoExtension.EncryptedKeyVersion>(
@@ -341,7 +402,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
                     KMS_CLIENT_ENC_KEY_CACHE_NUM_REFILL_THREADS_DEFAULT),
             new EncryptedQueueRefiller());
     authToken = new DelegationTokenAuthenticatedURL.Token();
-    dtService = new Text(providerUri.toString());
     LOG.info("KMSClientProvider for KMS url: {} delegation token service: {}" +
         " created.", kmsUrl, dtService);
   }
@@ -415,7 +475,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
         @Override
         public HttpURLConnection run() throws Exception {
           DelegationTokenAuthenticatedURL authUrl =
-              createKMSAuthenticatedURL();
+              new DelegationTokenAuthenticatedURL(configurator);
           return authUrl.openConnection(url, authToken, doAsUser);
         }
       });
@@ -875,7 +935,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       LOG.debug("Renewing delegation token {} with url:{}, as:{}",
           token, url, doAsUser);
       final DelegationTokenAuthenticatedURL authUrl =
-          createKMSAuthenticatedURL();
+          new DelegationTokenAuthenticatedURL(configurator);
       return getActualUgi().doAs(
           new PrivilegedExceptionAction<Long>() {
             @Override
@@ -907,7 +967,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
               LOG.debug("Cancelling delegation token {} with url:{}, as:{}",
                   dToken, url, doAsUser);
               final DelegationTokenAuthenticatedURL authUrl =
-                  createKMSAuthenticatedURL();
+                  new DelegationTokenAuthenticatedURL(configurator);
               authUrl.cancelDelegationToken(url, token, doAsUser);
               return null;
             }
@@ -959,17 +1019,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return token;
   }
 
-  @VisibleForTesting
-  DelegationTokenAuthenticatedURL createKMSAuthenticatedURL() {
-    return new DelegationTokenAuthenticatedURL(configurator) {
-      @Override
-      public org.apache.hadoop.security.token.Token<? extends TokenIdentifier>
-          getDelegationToken(URL url, Credentials creds) {
-        return selectKMSDelegationToken(creds);
-      }
-    };
-  }
-
   @Override
   public Token<?>[] addDelegationTokens(final String renewer,
       Credentials credentials) throws IOException {
@@ -978,7 +1027,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     if (token == null) {
       final URL url = createURL(null, null, null, null);
       final DelegationTokenAuthenticatedURL authUrl =
-          createKMSAuthenticatedURL();
+          new DelegationTokenAuthenticatedURL(configurator);
       try {
         final String doAsUser = getDoAsUser();
         token = getActualUgi().doAs(new PrivilegedExceptionAction<Token<?>>() {
@@ -992,16 +1041,9 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           }
         });
         if (token != null) {
-          if (KMSDelegationToken.TOKEN_KIND.equals(token.getKind())) {
-            // do not set service for legacy kind, for compatibility.
-            token.setService(dtService);
-          }
-          LOG.info("New token created: ({})", token);
+          LOG.debug("New token received: ({})", token);
           credentials.addToken(token.getService(), token);
-          Token<?> legacyToken = createAndAddLegacyToken(credentials, token);
-          tokens = legacyToken == null ?
-              new Token<?>[] {token} :
-              new Token<?>[] {token, legacyToken};
+          tokens = new Token<?>[] { token };
         } else {
           throw new IOException("Got NULL as delegation token");
         }
@@ -1018,75 +1060,13 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return tokens;
   }
 
-  /**
-   * If {@link CommonConfigurationKeysPublic#KMS_CLIENT_COPY_LEGACY_TOKEN_KEY}
-   * is true when creating the provider, then copy the passed-in token of
-   * {@link KMSDelegationToken#TOKEN_KIND} and create a new token of
-   * {@link KMSDelegationToken#TOKEN_LEGACY_KIND}, and add it to credentials.
-   *
-   * @return The legacy token, or null if one should not be created.
-   */
-  private Token<?> createAndAddLegacyToken(Credentials credentials,
-      Token<?> token) {
-    if (!copyLegacyToken || !KMSDelegationToken.TOKEN_KIND
-        .equals(token.getKind())) {
-      LOG.debug("Not creating legacy token because copyLegacyToken={}, "
-          + "token={}", copyLegacyToken, token);
-      return null;
-    }
-    // copy a KMS_DELEGATION_TOKEN and create a new kms-dt with the same
-    // underlying token for backwards-compatibility. Old clients/renewers
-    // does not parse the new token and can only work with kms-dt.
-    final Token<?> legacyToken = token.copyToken();
-    legacyToken.setKind(KMSDelegationToken.TOKEN_LEGACY_KIND);
-    final InetSocketAddress addr =
-        new InetSocketAddress(kmsUrl.getHost(), kmsUrl.getPort());
-    final Text fallBackServiceText = SecurityUtil.buildTokenService(addr);
-    legacyToken.setService(fallBackServiceText);
-    LOG.info("Copied token to legacy kind: {}", legacyToken);
-    credentials.addToken(legacyToken.getService(), legacyToken);
-    return legacyToken;
-  }
-
-  @VisibleForTesting
-  public Text getDelegationTokenService() {
-    return dtService;
-  }
-
-  /**
-   * Given a list of tokens, return the token that should be used for KMS
-   * authentication.
-   */
-  @VisibleForTesting
-  Token selectKMSDelegationToken(Credentials creds) {
-    // always look for TOKEN_KIND first
-    final TokenSelector<AbstractDelegationTokenIdentifier> tokenSelector =
-        new AbstractDelegationTokenSelector<AbstractDelegationTokenIdentifier>(
-            KMSDelegationToken.TOKEN_KIND) {
-        };
-    Token token = tokenSelector.selectToken(dtService, creds.getAllTokens());
-    LOG.debug("Searching service {} found token {}", dtService, token);
-    if (token != null) {
-      return token;
-    }
-
-    // fall back to look for token by service, regardless of kind.
-    // this is old behavior, keeping for compatibility reasons (for example,
-    // even if KMS server is new, if the job is submitted with an old kms
-    // client, job runners on new version should be able to find the token).
-    final InetSocketAddress addr =
-        new InetSocketAddress(kmsUrl.getHost(), kmsUrl.getPort());
-    final Text fallBackServiceText = SecurityUtil.buildTokenService(addr);
-    token = creds.getToken(fallBackServiceText);
-    LOG.debug("Selected delegation token {} using service:{}", token,
-        fallBackServiceText);
-    return token;
-  }
-
-  private boolean containsKmsDt(UserGroupInformation ugi) {
-    final Credentials creds = ugi.getCredentials();
+  private boolean containsKmsDt(UserGroupInformation ugi) throws IOException {
+    // Add existing credentials from the UGI, since provider is cached.
+    Credentials creds = ugi.getCredentials();
     if (!creds.getAllTokens().isEmpty()) {
-      final Token dToken = selectKMSDelegationToken(creds);
+      LOG.debug("Searching for token that matches service: {}", dtService);
+      org.apache.hadoop.security.token.Token<? extends TokenIdentifier>
+          dToken = creds.getToken(dtService);
       if (dToken != null) {
         return true;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSDelegationToken.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSDelegationToken.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSDelegationToken.java
index 2642e79..adeebf2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSDelegationToken.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSDelegationToken.java
@@ -27,10 +27,7 @@ import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier
 @InterfaceAudience.Private
 public final class KMSDelegationToken {
 
-  public static final String TOKEN_LEGACY_KIND_STR = "kms-dt";
-  public static final Text TOKEN_LEGACY_KIND = new Text(TOKEN_LEGACY_KIND_STR);
-
-  public static final String TOKEN_KIND_STR = "KMS_DELEGATION_TOKEN";
+  public static final String TOKEN_KIND_STR = "kms-dt";
   public static final Text TOKEN_KIND = new Text(TOKEN_KIND_STR);
 
   // Utility class is not supposed to be instantiated.
@@ -52,21 +49,4 @@ public final class KMSDelegationToken {
       return TOKEN_KIND;
     }
   }
-
-  /**
-   * DelegationTokenIdentifier used for the KMS for legacy tokens.
-   */
-  @Deprecated
-  public static class KMSLegacyDelegationTokenIdentifier
-      extends DelegationTokenIdentifier {
-
-    public KMSLegacyDelegationTokenIdentifier() {
-      super(TOKEN_LEGACY_KIND);
-    }
-
-    @Override
-    public Text getKind() {
-      return TOKEN_LEGACY_KIND;
-    }
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSLegacyTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSLegacyTokenRenewer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSLegacyTokenRenewer.java
deleted file mode 100644
index fd27073..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSLegacyTokenRenewer.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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.crypto.key.kms;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.KMSUtil;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_LEGACY_KIND;
-
-/**
- * The {@link KMSTokenRenewer} that supports legacy tokens.
- */
-@InterfaceAudience.Private
-@Deprecated
-public class KMSLegacyTokenRenewer extends KMSTokenRenewer {
-
-  @Override
-  public boolean handleKind(Text kind) {
-    return kind.equals(TOKEN_LEGACY_KIND);
-  }
-
-  /**
-   * Create a key provider for token renewal / cancellation.
-   * Caller is responsible for closing the key provider.
-   */
-  @Override
-  protected KeyProvider createKeyProvider(Token<?> token,
-      Configuration conf) throws IOException {
-    assert token.getKind().equals(TOKEN_LEGACY_KIND);
-    // Legacy tokens get service from configuration.
-    return KMSUtil.createKeyProvider(conf,
-        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSTokenRenewer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSTokenRenewer.java
deleted file mode 100644
index 1fff2f9..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSTokenRenewer.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * 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.crypto.key.kms;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenRenewer;
-import org.apache.hadoop.util.KMSUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_KIND;
-
-/**
- * The KMS implementation of {@link TokenRenewer}.
- */
-@InterfaceAudience.Private
-public class KMSTokenRenewer extends TokenRenewer {
-
-  public static final Logger LOG = LoggerFactory
-      .getLogger(org.apache.hadoop.crypto.key.kms.KMSTokenRenewer.class);
-
-  @Override
-  public boolean handleKind(Text kind) {
-    return kind.equals(TOKEN_KIND);
-  }
-
-  @Override
-  public boolean isManaged(Token<?> token) throws IOException {
-    return true;
-  }
-
-  @Override
-  public long renew(Token<?> token, Configuration conf) throws IOException {
-    LOG.debug("Renewing delegation token {}", token);
-    final KeyProvider keyProvider = createKeyProvider(token, conf);
-    try {
-      if (!(keyProvider instanceof
-          KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
-        throw new IOException(String
-            .format("keyProvider %s cannot renew token [%s]",
-                keyProvider == null ? "null" : keyProvider.getClass(), token));
-      }
-      return ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
-          keyProvider).renewDelegationToken(token);
-    } finally {
-      if (keyProvider != null) {
-        keyProvider.close();
-      }
-    }
-  }
-
-  @Override
-  public void cancel(Token<?> token, Configuration conf) throws IOException {
-    LOG.debug("Canceling delegation token {}", token);
-    final KeyProvider keyProvider = createKeyProvider(token, conf);
-    try {
-      if (!(keyProvider instanceof
-          KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
-        throw new IOException(String
-            .format("keyProvider %s cannot cancel token [%s]",
-                keyProvider == null ? "null" : keyProvider.getClass(), token));
-      }
-      ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
-          keyProvider).cancelDelegationToken(token);
-    } finally {
-      if (keyProvider != null) {
-        keyProvider.close();
-      }
-    }
-  }
-
-  /**
-   * Create a key provider for token renewal / cancellation.
-   * Caller is responsible for closing the key provider.
-   */
-  protected KeyProvider createKeyProvider(Token<?> token,
-      Configuration conf) throws IOException {
-    return KMSUtil
-        .createKeyProviderFromTokenService(conf, token.getService().toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/package-info.java
deleted file mode 100644
index eea93c2..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/package-info.java
+++ /dev/null
@@ -1,18 +0,0 @@
-/**
- * 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.crypto.key.kms;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index be1c7bc..8cd753a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -775,16 +775,6 @@ public class CommonConfigurationKeysPublic {
    * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
    * core-default.xml</a>
    */
-  public static final String KMS_CLIENT_COPY_LEGACY_TOKEN_KEY =
-      "hadoop.security.kms.client.copy.legacy.token";
-  /**  Default value is true. */
-  public static final boolean KMS_CLIENT_COPY_LEGACY_TOKEN_DEFAULT = true;
-
-  /**
-   * @see
-   * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
-   * core-default.xml</a>
-   */
   public static final String KMS_CLIENT_FAILOVER_SLEEP_MAX_MILLIS_KEY =
       "hadoop.security.kms.client.failover.sleep.max.millis";
   /** Default value is 2 secs. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java
index 0ddc4fc..0b1fdf8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java
@@ -300,7 +300,11 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
             creds.getAllTokens());
       }
       if (!creds.getAllTokens().isEmpty()) {
-        dToken = getDelegationToken(url, creds);
+        InetSocketAddress serviceAddr = new InetSocketAddress(url.getHost(),
+            url.getPort());
+        Text service = SecurityUtil.buildTokenService(serviceAddr);
+        dToken = creds.getToken(service);
+        LOG.debug("Using delegation token {} from service:{}", dToken, service);
         if (dToken != null) {
           if (useQueryStringForDelegationToken()) {
             // delegation token will go in the query string, injecting it
@@ -337,21 +341,6 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
   }
 
   /**
-   * Select a delegation token from all tokens in credentials, based on url.
-   */
-  @InterfaceAudience.Private
-  public org.apache.hadoop.security.token.Token<? extends TokenIdentifier>
-      getDelegationToken(URL url, Credentials creds) {
-    final InetSocketAddress serviceAddr =
-        new InetSocketAddress(url.getHost(), url.getPort());
-    final Text service = SecurityUtil.buildTokenService(serviceAddr);
-    org.apache.hadoop.security.token.Token<? extends TokenIdentifier> dToken =
-        creds.getToken(service);
-    LOG.debug("Selected delegation token {} using service:{}", dToken, service);
-    return dToken;
-  }
-
-  /**
    * Requests a delegation token using the configured <code>Authenticator</code>
    * for authentication.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
index 0ef102e7..6ee59f1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
@@ -81,7 +81,7 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceStability.Evolving
 public abstract class DelegationTokenAuthenticationHandler
     implements AuthenticationHandler {
-  public static final Logger LOG =
+  private static final Logger LOG =
       LoggerFactory.getLogger(DelegationTokenAuthenticationHandler.class);
 
   protected static final String TYPE_POSTFIX = "-dt";
@@ -224,8 +224,7 @@ public abstract class DelegationTokenAuthenticationHandler
       HttpServletRequest request, HttpServletResponse response)
       throws IOException, AuthenticationException {
     boolean requestContinues = true;
-    LOG.trace("Processing operation for req=({}), token: {}",
-        request.getRequestURL(), token);
+    LOG.trace("Processing operation for req=({}), token: {}", request, token);
     String op = ServletUtils.getParameter(request,
         KerberosDelegationTokenAuthenticator.OP_PARAM);
     op = (op != null) ? StringUtils.toUpperCase(op) : null;
@@ -408,8 +407,7 @@ public abstract class DelegationTokenAuthenticationHandler
             HttpServletResponse.SC_FORBIDDEN, new AuthenticationException(ex));
       }
     } else {
-      LOG.debug("Falling back to {} (req={})", authHandler.getClass(),
-          request.getRequestURL());
+      LOG.debug("Falling back to {} (req={})", authHandler.getClass(), request);
       token = authHandler.authenticate(request, response);
     }
     return token;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
index 7e83781..617773b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
@@ -50,7 +50,7 @@ import java.util.Map;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public abstract class DelegationTokenAuthenticator implements Authenticator {
-  public static final Logger LOG =
+  private static Logger LOG = 
       LoggerFactory.getLogger(DelegationTokenAuthenticator.class);
   
   private static final String CONTENT_TYPE = "Content-Type";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
index 8077076..c96c6fb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
@@ -30,7 +30,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.HashMap;
@@ -42,7 +41,8 @@ import java.util.Map;
  */
 @InterfaceAudience.Private
 public final class KMSUtil {
-  public static final Logger LOG = LoggerFactory.getLogger(KMSUtil.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(KMSUtil.class);
 
   private KMSUtil() { /* Hidden constructor */ }
 
@@ -64,13 +64,6 @@ public final class KMSUtil {
     if (providerUriStr == null || providerUriStr.isEmpty()) {
       return null;
     }
-    KeyProvider kp = KMSUtilFaultInjector.get().createKeyProviderForTests(
-        providerUriStr, conf);
-    if (kp != null) {
-      LOG.info("KeyProvider is created with uri: {}. This should happen only " +
-              "in tests.", providerUriStr);
-      return kp;
-    }
     return createKeyProviderFromUri(conf, URI.create(providerUriStr));
   }
 
@@ -212,38 +205,4 @@ public final class KMSUtil {
     }
     return metadata;
   }
-
-  /**
-   * Creates a key provider from token service field, which must be URI format.
-   *
-   * @param conf
-   * @param tokenServiceValue
-   * @return new KeyProvider or null
-   * @throws IOException
-   */
-  public static KeyProvider createKeyProviderFromTokenService(
-      final Configuration conf, final String tokenServiceValue)
-      throws IOException {
-    LOG.debug("Creating key provider from token service value {}. ",
-        tokenServiceValue);
-    final KeyProvider kp = KMSUtilFaultInjector.get()
-        .createKeyProviderForTests(tokenServiceValue, conf);
-    if (kp != null) {
-      LOG.info("KeyProvider is created with uri: {}. This should happen only "
-          + "in tests.", tokenServiceValue);
-      return kp;
-    }
-    if (!tokenServiceValue.contains("://")) {
-      throw new IllegalArgumentException(
-          "Invalid token service " + tokenServiceValue);
-    }
-    final URI tokenServiceUri;
-    try {
-      tokenServiceUri = new URI(tokenServiceValue);
-    } catch (URISyntaxException e) {
-      throw new IllegalArgumentException(
-          "Invalid token service " + tokenServiceValue, e);
-    }
-    return createKeyProviderFromUri(conf, tokenServiceUri);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtilFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtilFaultInjector.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtilFaultInjector.java
deleted file mode 100644
index 46d5069..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtilFaultInjector.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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.util;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
-
-import java.io.IOException;
-
-/**
- * Used for returning custom KeyProvider from test methods.
- */
-@VisibleForTesting
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class KMSUtilFaultInjector {
-  private static KMSUtilFaultInjector instance = new KMSUtilFaultInjector();
-
-  public static KMSUtilFaultInjector get() {
-    return instance;
-  }
-
-  public static void set(KMSUtilFaultInjector injector) {
-    instance = injector;
-  }
-
-  public KeyProvider createKeyProviderForTests(String value, Configuration conf)
-      throws IOException {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier b/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
index 43d06e2..b65f151 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
+++ b/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
@@ -12,4 +12,3 @@
 #   limitations under the License.
 #
 org.apache.hadoop.crypto.key.kms.KMSDelegationToken$KMSDelegationTokenIdentifier
-org.apache.hadoop.crypto.key.kms.KMSDelegationToken$KMSLegacyDelegationTokenIdentifier

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer b/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
index 5b6082c..56320fb 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
+++ b/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
@@ -11,5 +11,4 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 #
-org.apache.hadoop.crypto.key.kms.KMSTokenRenewer
-org.apache.hadoop.crypto.key.kms.KMSLegacyTokenRenewer
\ No newline at end of file
+org.apache.hadoop.crypto.key.kms.KMSClientProvider$KMSTokenRenewer
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 3a00131..dc13cb9 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -2616,26 +2616,6 @@
 </property>
 
 <property>
-  <name>hadoop.security.kms.client.copy.legacy.token</name>
-  <value>true</value>
-  <description>
-    Expert only. Whether the KMS client provider should copy a token to legacy
-    kind. This is for KMS_DELEGATION_TOKEN to be backwards compatible. With the
-    default value set to true, the client will locally duplicate the
-    KMS_DELEGATION_TOKEN token and create a kms-dt token, with the service field
-    conforming to kms-dt. All other parts of the token remain the same.
-    Then the new clients will use KMS_DELEGATION_TOKEN and old clients will
-    use kms-dt to authenticate. Default value is true.
-    You should only change this to false if you know all the KMS servers
-    , clients (including both job submitters and job runners) and the
-    token renewers (usually Yarn RM) are on a version that supports
-    KMS_DELEGATION_TOKEN.
-    Turning this off prematurely may result in old clients failing to
-    authenticate with new servers.
-  </description>
-</property>
-
-<property>
   <name>hadoop.security.kms.client.failover.sleep.max.millis</name>
   <value>2000</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestKMSClientProvider.java
deleted file mode 100644
index 56aace5..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestKMSClientProvider.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.crypto.key.kms;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.slf4j.event.Level;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.net.URL;
-
-import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_KIND;
-import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_LEGACY_KIND;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
-/**
- * Unit test for {@link KMSClientProvider} class.
- */
-public class TestKMSClientProvider {
-
-  public static final Logger LOG =
-      LoggerFactory.getLogger(TestKMSClientProvider.class);
-
-  private final Token token = new Token();
-  private final Token legacyToken = new Token();
-  private final String uriString = "kms://https@host:16000/kms";
-  private final String legacyTokenService = "host:16000";
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(30000);
-
-  {
-    GenericTestUtils.setLogLevel(KMSClientProvider.LOG, Level.TRACE);
-  }
-
-  @Before
-  public void setup() {
-    SecurityUtil.setTokenServiceUseIp(false);
-    token.setKind(TOKEN_KIND);
-    token.setService(new Text(uriString));
-    legacyToken.setKind(TOKEN_LEGACY_KIND);
-    legacyToken.setService(new Text(legacyTokenService));
-  }
-
-  @Test
-  public void testNotCopyFromLegacyToken() throws Exception {
-    final DelegationTokenAuthenticatedURL url =
-        mock(DelegationTokenAuthenticatedURL.class);
-    final Configuration conf = new Configuration();
-    final URI uri = new URI(uriString);
-    final KMSClientProvider kp = new KMSClientProvider(uri, conf, uri);
-    try {
-      final KMSClientProvider spyKp = spy(kp);
-      when(spyKp.createKMSAuthenticatedURL()).thenReturn(url);
-      when(url.getDelegationToken(any(URL.class),
-          any(DelegationTokenAuthenticatedURL.Token.class), any(String.class),
-          any(String.class))).thenReturn(legacyToken);
-
-      final Credentials creds = new Credentials();
-      final Token<?>[] tokens = spyKp.addDelegationTokens("yarn", creds);
-      LOG.info("Got tokens: {}", tokens);
-      assertEquals(1, tokens.length);
-      LOG.info("uri:" + uriString);
-      // if KMS server returned a legacy token, new client should leave the
-      // service being legacy and not set uri string
-      assertEquals(legacyTokenService, tokens[0].getService().toString());
-    } finally {
-      kp.close();
-    }
-  }
-
-  @Test
-  public void testCopyFromToken() throws Exception {
-    final DelegationTokenAuthenticatedURL url =
-        mock(DelegationTokenAuthenticatedURL.class);
-    final Configuration conf = new Configuration();
-    final URI uri = new URI(uriString);
-    final KMSClientProvider kp = new KMSClientProvider(uri, conf, uri);
-    try {
-      final KMSClientProvider spyKp = spy(kp);
-      when(spyKp.createKMSAuthenticatedURL()).thenReturn(url);
-      when(url.getDelegationToken(any(URL.class),
-          any(DelegationTokenAuthenticatedURL.Token.class), any(String.class),
-          any(String.class))).thenReturn(token);
-
-      final Credentials creds = new Credentials();
-      final Token<?>[] tokens = spyKp.addDelegationTokens("yarn", creds);
-      LOG.info("Got tokens: {}", tokens);
-      assertEquals(2, tokens.length);
-      assertTrue(creds.getAllTokens().contains(token));
-      assertNotNull(creds.getToken(legacyToken.getService()));
-    } finally {
-      kp.close();
-    }
-  }
-
-  @Test
-  public void testSelectTokenWhenBothExist() throws Exception {
-    final Credentials creds = new Credentials();
-    final Configuration conf = new Configuration();
-    final URI uri = new URI(uriString);
-    final KMSClientProvider kp = new KMSClientProvider(uri, conf, uri);
-    try {
-      creds.addToken(token.getService(), token);
-      creds.addToken(legacyToken.getService(), legacyToken);
-      Token t = kp.selectKMSDelegationToken(creds);
-      assertEquals(token, t);
-    } finally {
-      kp.close();
-    }
-  }
-
-  @Test
-  public void testSelectTokenLegacyService() throws Exception {
-    final Configuration conf = new Configuration();
-    final URI uri = new URI(uriString);
-    final KMSClientProvider kp = new KMSClientProvider(uri, conf, uri);
-    try {
-      Text legacyService = new Text(legacyTokenService);
-      token.setService(legacyService);
-      final Credentials creds = new Credentials();
-      creds.addToken(legacyService, token);
-      Token t = kp.selectKMSDelegationToken(creds);
-      assertEquals(token, t);
-    } finally {
-      kp.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
index e6a9fe0..bd68dca 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
@@ -42,8 +42,7 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.hadoop.util.KMSUtil;
-import org.apache.hadoop.util.KMSUtilFaultInjector;
+import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -57,68 +56,33 @@ public class TestLoadBalancingKMSClientProvider {
     SecurityUtil.setTokenServiceUseIp(false);
   }
 
-  private void setKMSUtilFaultInjector() {
-    KMSUtilFaultInjector injector = new KMSUtilFaultInjector() {
-      @Override
-      public KeyProvider createKeyProviderForTests(
-          String value, Configuration conf) throws IOException {
-        return TestLoadBalancingKMSClientProvider
-            .createKeyProviderForTests(value, conf);
-      }
-    };
-    KMSUtilFaultInjector.set(injector);
-  }
-
-  public static KeyProvider createKeyProviderForTests(
-      String value, Configuration conf) throws IOException {
-    // The syntax for kms servers will be
-    // kms://http@localhost:port1/kms,kms://http@localhost:port2/kms
-    if (!value.contains(",")) {
-      return null;
-    }
-    String[] keyProviderUrisStr = value.split(",");
-    KMSClientProvider[] keyProviderArr =
-        new KMSClientProvider[keyProviderUrisStr.length];
-
-    int i = 0;
-    for (String keyProviderUri: keyProviderUrisStr) {
-      KMSClientProvider kmcp =
-          new KMSClientProvider(URI.create(keyProviderUri), conf, URI
-              .create(value));
-      keyProviderArr[i] = kmcp;
-      i++;
-    }
-    LoadBalancingKMSClientProvider lbkcp =
-        new LoadBalancingKMSClientProvider(keyProviderArr, conf);
-    return lbkcp;
+  @After
+  public void teardown() throws IOException {
+    KMSClientProvider.fallbackDefaultPortForTesting = false;
   }
 
   @Test
   public void testCreation() throws Exception {
     Configuration conf = new Configuration();
+    KMSClientProvider.fallbackDefaultPortForTesting = true;
     KeyProvider kp = new KMSClientProvider.Factory().createProvider(new URI(
-        "kms://http@host1:9600/kms/foo"), conf);
+        "kms://http@host1/kms/foo"), conf);
     assertTrue(kp instanceof LoadBalancingKMSClientProvider);
     KMSClientProvider[] providers =
         ((LoadBalancingKMSClientProvider) kp).getProviders();
     assertEquals(1, providers.length);
-    assertEquals(Sets.newHashSet("http://host1:9600/kms/foo/v1/"),
+    assertEquals(Sets.newHashSet("http://host1/kms/foo/v1/"),
         Sets.newHashSet(providers[0].getKMSUrl()));
-    setKMSUtilFaultInjector();
-    String uriStr = "kms://http@host1:9600/kms/foo," +
-        "kms://http@host2:9600/kms/foo," +
-        "kms://http@host3:9600/kms/foo";
-    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
-        uriStr);
-    kp = KMSUtil.createKeyProvider(conf, CommonConfigurationKeysPublic
-        .HADOOP_SECURITY_KEY_PROVIDER_PATH);
+
+    kp = new KMSClientProvider.Factory().createProvider(new URI(
+        "kms://http@host1;host2;host3/kms/foo"), conf);
     assertTrue(kp instanceof LoadBalancingKMSClientProvider);
     providers =
         ((LoadBalancingKMSClientProvider) kp).getProviders();
     assertEquals(3, providers.length);
-    assertEquals(Sets.newHashSet("http://host1:9600/kms/foo/v1/",
-        "http://host2:9600/kms/foo/v1/",
-        "http://host3:9600/kms/foo/v1/"),
+    assertEquals(Sets.newHashSet("http://host1/kms/foo/v1/",
+        "http://host2/kms/foo/v1/",
+        "http://host3/kms/foo/v1/"),
         Sets.newHashSet(providers[0].getKMSUrl(),
             providers[1].getKMSUrl(),
             providers[2].getKMSUrl()));
@@ -244,7 +208,7 @@ public class TestLoadBalancingKMSClientProvider {
 
   private class MyKMSClientProvider extends KMSClientProvider {
     public MyKMSClientProvider(URI uri, Configuration conf) throws IOException {
-      super(uri, conf, uri);
+      super(uri, conf);
     }
 
     @Override
@@ -281,8 +245,9 @@ public class TestLoadBalancingKMSClientProvider {
   @Test
   public void testClassCastException() throws Exception {
     Configuration conf = new Configuration();
+    KMSClientProvider.fallbackDefaultPortForTesting = true;
     KMSClientProvider p1 = new MyKMSClientProvider(
-        new URI("kms://http@host1:9600/kms/foo"), conf);
+        new URI("kms://http@host1/kms/foo"), conf);
     LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
         new KMSClientProvider[] {p1}, 0, conf);
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a1552c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestKMSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestKMSUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestKMSUtil.java
deleted file mode 100644
index 77f52ee..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestKMSUtil.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * 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.util;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test {@link KMSUtil}.
- */
-public class TestKMSUtil {
-
-  public static final Logger LOG = LoggerFactory.getLogger(TestKMSUtil.class);
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(90000);
-
-  @Test
-  public void testCreateKeyProviderFromTokenService() throws Exception {
-    final Configuration conf = new Configuration();
-    KeyProvider kp = KMSUtil.createKeyProviderFromTokenService(conf,
-        "kms://https@localhost:9600/kms");
-    assertNotNull(kp);
-    kp.close();
-
-    kp = KMSUtil.createKeyProviderFromTokenService(conf,
-        "kms://https@localhost:9600/kms,kms://localhost1:9600/kms");
-    assertNotNull(kp);
-    kp.close();
-
-    String invalidService = "whatever:9600";
-    try {
-      KMSUtil.createKeyProviderFromTokenService(conf, invalidService);
-    } catch (Exception ex) {
-      LOG.info("Expected exception:", ex);
-      assertTrue(ex instanceof IllegalArgumentException);
-      GenericTestUtils.assertExceptionContains(
-          "Invalid token service " + invalidService, ex);
-    }
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/35] hadoop git commit: YARN-8079. Support static and archive unmodified local resources in service AM. Contributed by Suma Shivaprasad

Posted by xy...@apache.org.
YARN-8079. Support static and archive unmodified local resources in service AM. Contributed by Suma Shivaprasad


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6795f807
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6795f807
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6795f807

Branch: refs/heads/HDDS-4
Commit: 6795f8072ffbe6138857e77d51af173f33e4e5c1
Parents: 502914c
Author: Billie Rinaldi <bi...@apache.org>
Authored: Fri May 4 09:27:07 2018 -0700
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Fri May 4 09:27:07 2018 -0700

----------------------------------------------------------------------
 ...RN-Simplified-V1-API-Layer-For-Services.yaml |   2 +
 .../yarn/service/api/records/ConfigFile.java    |   3 +-
 .../yarn/service/conf/YarnServiceConstants.java |   1 +
 .../provider/AbstractClientProvider.java        |  23 ++-
 .../provider/AbstractProviderService.java       |   4 +
 .../yarn/service/provider/ProviderUtils.java    |  91 ++++++++--
 .../service/provider/TestProviderUtils.java     | 164 +++++++++++++++++++
 .../providers/TestAbstractClientProvider.java   |  44 +++++
 .../markdown/yarn-service/YarnServiceAPI.md     |   4 +-
 9 files changed, 321 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
index 8c5ad65..cea8296 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
@@ -475,6 +475,8 @@ definitions:
           - YAML
           - TEMPLATE
           - HADOOP_XML
+          - STATIC
+          - ARCHIVE
       dest_file:
         type: string
         description: The path that this configuration file should be created as. If it is an absolute path, it will be mounted into the DOCKER container. Absolute paths are only allowed for DOCKER containers.  If it is a relative path, only the file name should be provided, and the file will be created in the container local working directory under a folder named conf.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java
index d3b18bc..623feed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java
@@ -55,7 +55,8 @@ public class ConfigFile implements Serializable {
   @XmlEnum
   public enum TypeEnum {
     XML("XML"), PROPERTIES("PROPERTIES"), JSON("JSON"), YAML("YAML"), TEMPLATE(
-        "TEMPLATE"), HADOOP_XML("HADOOP_XML");
+        "TEMPLATE"), HADOOP_XML("HADOOP_XML"), STATIC("STATIC"), ARCHIVE(
+        "ARCHIVE");
 
     private String value;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java
index 7b474f6..d081606 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java
@@ -84,6 +84,7 @@ public interface YarnServiceConstants {
   String HADOOP_USER_NAME = "HADOOP_USER_NAME";
 
   String APP_CONF_DIR = "conf";
+  String APP_RESOURCES_DIR = "resources";
 
   String APP_LIB_DIR = "lib";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
index 26c332b..d16c698 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service.provider;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
@@ -86,8 +87,9 @@ public abstract class AbstractClientProvider {
       if (file.getType() == null) {
         throw new IllegalArgumentException("File type is empty");
       }
+      ConfigFile.TypeEnum fileType = file.getType();
 
-      if (file.getType().equals(ConfigFile.TypeEnum.TEMPLATE)) {
+      if (fileType.equals(ConfigFile.TypeEnum.TEMPLATE)) {
         if (StringUtils.isEmpty(file.getSrcFile()) &&
             !file.getProperties().containsKey(CONTENT)) {
           throw new IllegalArgumentException(MessageFormat.format("For {0} " +
@@ -96,6 +98,25 @@ public abstract class AbstractClientProvider {
                   "the 'properties' field of ConfigFile. ",
               ConfigFile.TypeEnum.TEMPLATE, CONTENT));
         }
+      } else if (fileType.equals(ConfigFile.TypeEnum.STATIC) || fileType.equals(
+          ConfigFile.TypeEnum.ARCHIVE)) {
+        if (!file.getProperties().isEmpty()) {
+          throw new IllegalArgumentException(String
+              .format("For %s format, should not specify any 'properties.'",
+                  fileType));
+        }
+
+        String srcFile = file.getSrcFile();
+        if (srcFile == null || srcFile.isEmpty()) {
+          throw new IllegalArgumentException(String.format(
+              "For %s format, should make sure that srcFile is specified",
+              fileType));
+        }
+        FileStatus fileStatus = fs.getFileStatus(new Path(srcFile));
+        if (fileStatus != null && fileStatus.isDirectory()) {
+          throw new IllegalArgumentException("srcFile=" + srcFile +
+              " is a directory, which is not supported.");
+        }
       }
       if (!StringUtils.isEmpty(file.getSrcFile())) {
         Path p = new Path(file.getSrcFile());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java
index 560f421..5a17817 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java
@@ -97,6 +97,10 @@ public abstract class AbstractProviderService implements ProviderService,
     ProviderUtils.createConfigFileAndAddLocalResource(launcher, fileSystem,
         compLaunchContext, tokensForSubstitution, instance, context);
 
+    // handles static files (like normal file / archive file) for localization.
+    ProviderUtils.handleStaticFilesForLocalization(launcher, fileSystem,
+        compLaunchContext);
+
     // substitute launch command
     String launchCommand = compLaunchContext.getLaunchCommand();
     // docker container may have empty commands

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
index 2fc8cfb..1ad5fd8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service.provider;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -198,6 +199,10 @@ public class ProviderUtils implements YarnServiceConstants {
 
     for (ConfigFile originalFile : compLaunchContext.getConfiguration()
         .getFiles()) {
+
+      if (isStaticFile(originalFile)) {
+        continue;
+      }
       ConfigFile configFile = originalFile.copy();
       String fileName = new Path(configFile.getDestFile()).getName();
 
@@ -207,7 +212,14 @@ public class ProviderUtils implements YarnServiceConstants {
             .replaceAll(Pattern.quote(token.getKey()), token.getValue()));
       }
 
+      /* When source file is not specified, write new configs
+       * to compInstanceDir/fileName
+       * When source file is specified, it reads and performs variable
+       * substitution and merges in new configs, and writes a new file to
+       * compInstanceDir/fileName.
+       */
       Path remoteFile = new Path(compInstanceDir, fileName);
+
       if (!fs.getFileSystem().exists(remoteFile)) {
         log.info("Saving config file on hdfs for component " + instance
             .getCompInstanceName() + ": " + configFile);
@@ -239,22 +251,79 @@ public class ProviderUtils implements YarnServiceConstants {
       // Add resource for localization
       LocalResource configResource =
           fs.createAmResource(remoteFile, LocalResourceType.FILE);
-      File destFile = new File(configFile.getDestFile());
+      Path destFile = new Path(configFile.getDestFile());
       String symlink = APP_CONF_DIR + "/" + fileName;
-      if (destFile.isAbsolute()) {
-        launcher.addLocalResource(symlink, configResource,
-            configFile.getDestFile());
-        log.info("Add config file for localization: " + symlink + " -> "
-            + configResource.getResource().getFile() + ", dest mount path: "
-            + configFile.getDestFile());
-      } else {
-        launcher.addLocalResource(symlink, configResource);
-        log.info("Add config file for localization: " + symlink + " -> "
-            + configResource.getResource().getFile());
+      addLocalResource(launcher, symlink, configResource, destFile);
+    }
+  }
+
+  public static synchronized void handleStaticFilesForLocalization(
+      AbstractLauncher launcher, SliderFileSystem fs, ContainerLaunchService
+      .ComponentLaunchContext componentLaunchCtx)
+      throws IOException {
+    for (ConfigFile staticFile :
+        componentLaunchCtx.getConfiguration().getFiles()) {
+      // Only handle static file here.
+      if (!isStaticFile(staticFile)) {
+        continue;
+      }
+
+      if (staticFile.getSrcFile() == null) {
+        // This should not happen, AbstractClientProvider should have checked
+        // this.
+        throw new IOException("srcFile is null, please double check.");
+      }
+      Path sourceFile = new Path(staticFile.getSrcFile());
+
+      // Output properties to sourceFile if not existed
+      if (!fs.getFileSystem().exists(sourceFile)) {
+        throw new IOException(
+            "srcFile=" + sourceFile + " doesn't exist, please double check.");
       }
+
+      FileStatus fileStatus = fs.getFileSystem().getFileStatus(sourceFile);
+      if (fileStatus.isDirectory()) {
+        throw new IOException("srcFile=" + sourceFile +
+            " is a directory, which is not supported.");
+      }
+
+      // Add resource for localization
+      LocalResource localResource = fs.createAmResource(sourceFile,
+          (staticFile.getType() == ConfigFile.TypeEnum.ARCHIVE ?
+              LocalResourceType.ARCHIVE :
+              LocalResourceType.FILE));
+      Path destFile = new Path(sourceFile.getName());
+      if (staticFile.getDestFile() != null && !staticFile.getDestFile()
+          .isEmpty()) {
+        destFile = new Path(staticFile.getDestFile());
+      }
+
+      String symlink = APP_RESOURCES_DIR + "/" + destFile.getName();
+      addLocalResource(launcher, symlink, localResource, destFile);
     }
   }
 
+  private static void addLocalResource(AbstractLauncher launcher,
+      String symlink, LocalResource localResource, Path destFile) {
+    if (destFile.isAbsolute()) {
+      launcher.addLocalResource(symlink, localResource, destFile.toString());
+      log.info("Added file for localization: "+ symlink +" -> " +
+          localResource.getResource().getFile() + ", dest mount path: " +
+          destFile);
+    } else{
+      launcher.addLocalResource(symlink, localResource);
+      log.info("Added file for localization: " + symlink+ " -> " +
+          localResource.getResource().getFile());
+    }
+  }
+
+  // Static file is files uploaded by users before launch the service. Which
+  // should be localized to container local disk without any changes.
+  private static boolean isStaticFile(ConfigFile file) {
+    return file.getType().equals(ConfigFile.TypeEnum.ARCHIVE) || file.getType()
+        .equals(ConfigFile.TypeEnum.STATIC);
+  }
+
   private static void resolvePropsInConfigFileAndSaveOnHdfs(SliderFileSystem fs,
       Map<String, String> tokensForSubstitution, ComponentInstance instance,
       ConfigFile configFile, String fileName, Path remoteFile)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/provider/TestProviderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/provider/TestProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/provider/TestProviderUtils.java
new file mode 100644
index 0000000..6e8bc43
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/provider/TestProviderUtils.java
@@ -0,0 +1,164 @@
+/*
+ * 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.yarn.service.provider;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.service.api.records.ConfigFile;
+import org.apache.hadoop.yarn.service.api.records.Configuration;
+import org.apache.hadoop.yarn.service.containerlaunch.AbstractLauncher;
+import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService;
+import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test functionality of ProviderUtils.
+ */
+public class TestProviderUtils {
+  @Test
+  public void testStaticFileLocalization() throws IOException {
+    // A bunch of mocks ...
+    ContainerLaunchService.ComponentLaunchContext compLaunchCtx =
+        mock(ContainerLaunchService.ComponentLaunchContext.class);
+    AbstractLauncher launcher = mock(AbstractLauncher.class);
+    SliderFileSystem sfs = mock(SliderFileSystem.class);
+    FileSystem fs = mock(FileSystem.class);
+    when(fs.getFileStatus(any(Path.class))).thenAnswer(
+        invocationOnMock -> new FileStatus(1L, false, 1, 1L, 1L,
+            (Path) invocationOnMock.getArguments()[0]));
+    when(fs.exists(any(Path.class))).thenReturn(true);
+    when(sfs.getFileSystem()).thenReturn(fs);
+    Configuration conf = mock(Configuration.class);
+    List<ConfigFile> configFileList = new ArrayList<>();
+    when(conf.getFiles()).thenReturn(configFileList);
+    when(compLaunchCtx.getConfiguration()).thenReturn(conf);
+    when(sfs.createAmResource(any(Path.class), any(LocalResourceType.class)))
+        .thenAnswer(invocationOnMock -> new LocalResource() {
+          @Override
+          public URL getResource() {
+            return URL.fromPath(((Path) invocationOnMock.getArguments()[0]));
+          }
+
+          @Override
+          public void setResource(URL resource) {
+
+          }
+
+          @Override
+          public long getSize() {
+            return 0;
+          }
+
+          @Override
+          public void setSize(long size) {
+
+          }
+
+          @Override
+          public long getTimestamp() {
+            return 0;
+          }
+
+          @Override
+          public void setTimestamp(long timestamp) {
+
+          }
+
+          @Override
+          public LocalResourceType getType() {
+            return (LocalResourceType) invocationOnMock.getArguments()[1];
+          }
+
+          @Override
+          public void setType(LocalResourceType type) {
+
+          }
+
+          @Override
+          public LocalResourceVisibility getVisibility() {
+            return null;
+          }
+
+          @Override
+          public void setVisibility(LocalResourceVisibility visibility) {
+
+          }
+
+          @Override
+          public String getPattern() {
+            return null;
+          }
+
+          @Override
+          public void setPattern(String pattern) {
+
+          }
+
+          @Override
+          public boolean getShouldBeUploadedToSharedCache() {
+            return false;
+          }
+
+          @Override
+          public void setShouldBeUploadedToSharedCache(
+              boolean shouldBeUploadedToSharedCache) {
+
+          }
+        });
+
+    // Initialize list of files.
+    //archive
+    configFileList.add(new ConfigFile().srcFile("hdfs://default/sourceFile1")
+        .destFile("destFile1").type(ConfigFile.TypeEnum.ARCHIVE));
+
+    //static file
+    configFileList.add(new ConfigFile().srcFile("hdfs://default/sourceFile2")
+        .destFile("folder/destFile_2").type(ConfigFile.TypeEnum.STATIC));
+
+    //This will be ignored since type is JSON
+    configFileList.add(new ConfigFile().srcFile("hdfs://default/sourceFile3")
+        .destFile("destFile3").type(ConfigFile.TypeEnum.JSON));
+    //No destination file specified
+    configFileList.add(new ConfigFile().srcFile("hdfs://default/sourceFile4")
+        .type(ConfigFile.TypeEnum.STATIC));
+
+    ProviderUtils.handleStaticFilesForLocalization(launcher, sfs,
+        compLaunchCtx);
+    Mockito.verify(launcher).addLocalResource(Mockito.eq("resources/destFile1"),
+        any(LocalResource.class));
+    Mockito.verify(launcher).addLocalResource(
+        Mockito.eq("resources/destFile_2"), any(LocalResource.class));
+    Mockito.verify(launcher).addLocalResource(
+        Mockito.eq("resources/sourceFile4"), any(LocalResource.class));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
index 79406e9..1d6be91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.yarn.service.providers;
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
 import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
@@ -57,6 +59,7 @@ public class TestAbstractClientProvider {
   public void testConfigFiles() throws IOException {
     ClientProvider clientProvider = new ClientProvider();
     FileSystem mockFs = mock(FileSystem.class);
+    FileStatus mockFileStatus = mock(FileStatus.class);
     when(mockFs.exists(anyObject())).thenReturn(true);
 
     ConfigFile configFile = new ConfigFile();
@@ -114,5 +117,46 @@ public class TestAbstractClientProvider {
       Assert.fail(EXCEPTION_PREFIX + "duplicate dest file");
     } catch (IllegalArgumentException e) {
     }
+
+    configFiles.clear();
+    configFile = new ConfigFile();
+    configFile.setType(ConfigFile.TypeEnum.STATIC);
+    configFile.setSrcFile(null);
+    configFile.setDestFile("path/destfile3");
+    configFiles.add(configFile);
+    try {
+      clientProvider.validateConfigFiles(configFiles, mockFs);
+      Assert.fail(EXCEPTION_PREFIX + "dest file with multiple path elements");
+    } catch (IllegalArgumentException e) {
+    }
+
+    configFile.setDestFile("/path/destfile3");
+    try {
+      clientProvider.validateConfigFiles(configFiles, mockFs);
+      Assert.fail(EXCEPTION_PREFIX + "src file should be specified");
+    } catch (IllegalArgumentException e) {
+    }
+
+    //should succeed
+    configFile.setSrcFile("srcFile");
+    configFile.setDestFile("destfile3");
+    clientProvider.validateConfigFiles(configFiles, mockFs);
+
+    when(mockFileStatus.isDirectory()).thenReturn(true);
+    when(mockFs.getFileStatus(new Path("srcFile")))
+        .thenReturn(mockFileStatus).thenReturn(mockFileStatus);
+
+    configFiles.clear();
+    configFile = new ConfigFile();
+    configFile.setType(ConfigFile.TypeEnum.STATIC);
+    configFile.setSrcFile("srcFile");
+    configFile.setDestFile("destfile3");
+    configFiles.add(configFile);
+
+    try {
+      clientProvider.validateConfigFiles(configFiles, mockFs);
+      Assert.fail(EXCEPTION_PREFIX + "src file is a directory");
+    } catch (IllegalArgumentException e) {
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6795f807/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
index fab33c5..c648046 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
@@ -250,8 +250,8 @@ A config file that needs to be created and made available as a volume in a servi
 
 |Name|Description|Required|Schema|Default|
 |----|----|----|----|----|
-|type|Config file in the standard format like xml, properties, json, yaml, template.|false|enum (XML, PROPERTIES, JSON, YAML, TEMPLATE, HADOOP_XML)||
-|dest_file|The path that this configuration file should be created as. If it is an absolute path, it will be mounted into the DOCKER container. Absolute paths are only allowed for DOCKER containers.  If it is a relative path, only the file name should be provided, and the file will be created in the container local working directory under a folder named conf.|false|string||
+|type|Config file in the standard format like xml, properties, json, yaml, template or static/archive resource files. When static/archive types are specified, file must be uploaded to remote file system before launching the job, and YARN service framework will localize files prior to launching containers. Archive files are unwrapped during localization |false|enum (XML, PROPERTIES, JSON, YAML, TEMPLATE, ENV, HADOOP_XML, STATIC, ARCHIVE)||
+|dest_file|The path that this configuration file should be created as. If it is an absolute path, it will be mounted into the DOCKER container. Absolute paths are only allowed for DOCKER containers.  If it is a relative path, only the file name should be provided, and the file will be created in the container local working directory under a folder named conf for all types other than static/archive. For static/archive resource types, the files are available under resources directory.|false|string||
 |src_file|This provides the source location of the configuration file, the content of which is dumped to dest_file post property substitutions, in the format as specified in type. Typically the src_file would point to a source controlled network accessible file maintained by tools like puppet, chef, or hdfs etc. Currently, only hdfs is supported.|false|string||
 |properties|A blob of key value pairs that will be dumped in the dest_file in the format as specified in type. If src_file is specified, src_file content are dumped in the dest_file and these properties will overwrite, if any, existing properties in src_file or be added as new properties in src_file.|false|object||
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/35] hadoop git commit: MAPREDUCE-7093. Use assertEquals instead of assertTrue(a == b) in TestMapReduceJobControlWithMocks

Posted by xy...@apache.org.
MAPREDUCE-7093. Use assertEquals instead of assertTrue(a == b) in TestMapReduceJobControlWithMocks

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e9159db6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e9159db6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e9159db6

Branch: refs/heads/HDDS-4
Commit: e9159db62367920c34616179aa4ced46d1ed22e1
Parents: a732acd
Author: Abhishek Modi <ab...@gmail.com>
Authored: Mon May 7 01:15:14 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon May 7 01:15:14 2018 +0900

----------------------------------------------------------------------
 .../TestMapReduceJobControlWithMocks.java        | 19 +++++++++----------
 1 file changed, 9 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9159db6/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java
index e89f090..a0d5125 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapreduce.lib.jobcontrol;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -52,10 +51,10 @@ public class TestMapReduceJobControlWithMocks {
     assertEquals("Success list", 4, jobControl.getSuccessfulJobList().size());
     assertEquals("Failed list", 0, jobControl.getFailedJobList().size());
     
-    assertTrue(job1.getJobState() == ControlledJob.State.SUCCESS);
-    assertTrue(job2.getJobState() == ControlledJob.State.SUCCESS);
-    assertTrue(job3.getJobState() == ControlledJob.State.SUCCESS);
-    assertTrue(job4.getJobState() == ControlledJob.State.SUCCESS);
+    assertEquals(ControlledJob.State.SUCCESS, job1.getJobState());
+    assertEquals(ControlledJob.State.SUCCESS, job2.getJobState());
+    assertEquals(ControlledJob.State.SUCCESS, job3.getJobState());
+    assertEquals(ControlledJob.State.SUCCESS, job4.getJobState());
     
     jobControl.stop();
   }
@@ -74,10 +73,10 @@ public class TestMapReduceJobControlWithMocks {
     assertEquals("Success list", 1, jobControl.getSuccessfulJobList().size());
     assertEquals("Failed list", 3, jobControl.getFailedJobList().size());
 
-    assertTrue(job1.getJobState() == ControlledJob.State.FAILED);
-    assertTrue(job2.getJobState() == ControlledJob.State.SUCCESS);
-    assertTrue(job3.getJobState() == ControlledJob.State.DEPENDENT_FAILED);
-    assertTrue(job4.getJobState() == ControlledJob.State.DEPENDENT_FAILED);
+    assertEquals(ControlledJob.State.FAILED, job1.getJobState());
+    assertEquals(ControlledJob.State.SUCCESS, job2.getJobState());
+    assertEquals(ControlledJob.State.DEPENDENT_FAILED, job3.getJobState());
+    assertEquals(ControlledJob.State.DEPENDENT_FAILED, job4.getJobState());
     
     jobControl.stop();
   }
@@ -99,7 +98,7 @@ public class TestMapReduceJobControlWithMocks {
       assertEquals("Success list", 0, jobControl.getSuccessfulJobList().size());
       assertEquals("Failed list", 1, jobControl.getFailedJobList().size());
 
-      assertTrue(job1.getJobState() == ControlledJob.State.FAILED);
+      assertEquals(ControlledJob.State.FAILED, job1.getJobState());
     } finally {
       jobControl.stop();
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/35] hadoop git commit: HDFS-13525. RBF: Add unit test TestStateStoreDisabledNameservice. Contributed by Yiqun Lin.

Posted by xy...@apache.org.
HDFS-13525. RBF: Add unit test TestStateStoreDisabledNameservice. Contributed by Yiqun Lin.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a3b416f6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a3b416f6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a3b416f6

Branch: refs/heads/HDDS-4
Commit: a3b416f69dc3965f247603f657df33bd74fd723e
Parents: 7698737
Author: Inigo Goiri <in...@apache.org>
Authored: Thu May 3 11:24:57 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Thu May 3 11:24:57 2018 -0700

----------------------------------------------------------------------
 .../TestStateStoreDisabledNameservice.java      | 71 ++++++++++++++++++++
 1 file changed, 71 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3b416f6/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreDisabledNameservice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreDisabledNameservice.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreDisabledNameservice.java
new file mode 100644
index 0000000..353510a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreDisabledNameservice.java
@@ -0,0 +1,71 @@
+/**
+ * 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.hdfs.server.federation.store;
+
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.clearRecords;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.hadoop.hdfs.server.federation.store.records.DisabledNameservice;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the basic {@link StateStoreService}
+ * {@link DisabledNameserviceStore} functionality.
+ */
+public class TestStateStoreDisabledNameservice extends TestStateStoreBase {
+
+  private static DisabledNameserviceStore disabledStore;
+
+  @Before
+  public void setup() throws IOException, InterruptedException {
+    disabledStore = getStateStore()
+        .getRegisteredRecordStore(DisabledNameserviceStore.class);
+    // Clear disabled nameservice registrations
+    assertTrue(clearRecords(getStateStore(), DisabledNameservice.class));
+  }
+
+  @Test
+  public void testDisableNameservice() throws IOException {
+    // no nameservices disabled firstly
+    Set<String> disabledNameservices = disabledStore.getDisabledNameservices();
+    assertEquals(0, disabledNameservices.size());
+
+    // disable two nameservices
+    disabledStore.disableNameservice("ns0");
+    disabledStore.disableNameservice("ns1");
+    disabledStore.loadCache(true);
+    // verify if the nameservices are disabled
+    disabledNameservices = disabledStore.getDisabledNameservices();
+    assertEquals(2, disabledNameservices.size());
+    assertTrue(disabledNameservices.contains("ns0")
+        && disabledNameservices.contains("ns1"));
+
+    // enable one nameservice
+    disabledStore.enableNameservice("ns0");
+    disabledStore.loadCache(true);
+    // verify the disabled nameservice again
+    disabledNameservices = disabledStore.getDisabledNameservices();
+    assertEquals(1, disabledNameservices.size());
+    assertTrue(disabledNameservices.contains("ns1"));
+  }
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/35] hadoop git commit: HADOOP-15250. Split-DNS MultiHomed Server Network Cluster Network IPC Client Bind Addr Wrong Contributed by Ajay Kumar

Posted by xy...@apache.org.
HADOOP-15250. Split-DNS MultiHomed Server Network Cluster Network IPC Client Bind Addr Wrong
Contributed by Ajay Kumar


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8f42dafc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8f42dafc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8f42dafc

Branch: refs/heads/HDDS-4
Commit: 8f42dafcf82d5b426dd931dc5ddd177dd6f283f7
Parents: 68c6ec7
Author: Steve Loughran <st...@apache.org>
Authored: Tue May 1 22:32:40 2018 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue May 1 22:32:40 2018 +0100

----------------------------------------------------------------------
 .../apache/hadoop/fs/CommonConfigurationKeys.java   |  4 ++++
 .../src/main/java/org/apache/hadoop/ipc/Client.java | 16 ++++++++++++----
 .../main/java/org/apache/hadoop/net/NetUtils.java   | 16 ++++++++++++++++
 .../src/main/resources/core-default.xml             |  8 ++++++++
 .../java/org/apache/hadoop/net/TestNetUtils.java    |  8 ++++++++
 5 files changed, 48 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f42dafc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 043e52a..1eb27f8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -341,6 +341,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String  IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "ipc.client.fallback-to-simple-auth-allowed";
   public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
 
+  public static final String  IPC_CLIENT_BIND_WILDCARD_ADDR_KEY = "ipc.client"
+      + ".bind.wildcard.addr";
+  public static final boolean IPC_CLIENT_BIND_WILDCARD_ADDR_DEFAULT = false;
+
   public static final String IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY =
     "ipc.client.connect.max.retries.on.sasl";
   public static final int    IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_DEFAULT = 5;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f42dafc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index a0417d6..163e80d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -135,6 +135,7 @@ public class Client implements AutoCloseable {
   private final int connectionTimeout;
 
   private final boolean fallbackAllowed;
+  private final boolean bindToWildCardAddress;
   private final byte[] clientId;
   private final int maxAsyncCalls;
   private final AtomicInteger asyncCallCounter = new AtomicInteger(0);
@@ -674,10 +675,10 @@ public class Client implements AutoCloseable {
               InetAddress localAddr = NetUtils.getLocalInetAddress(host);
               if (localAddr != null) {
                 this.socket.setReuseAddress(true);
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug("Binding " + principal + " to " + localAddr);
-                }
-                bindAddr = new InetSocketAddress(localAddr, 0);
+                localAddr = NetUtils.bindToLocalAddress(localAddr,
+                    bindToWildCardAddress);
+                LOG.debug("Binding {} to {}", principal, localAddr);
+                this.socket.bind(new InetSocketAddress(localAddr, 0));
               }
             }
           }
@@ -1277,6 +1278,13 @@ public class Client implements AutoCloseable {
         CommonConfigurationKeys.IPC_CLIENT_CONNECT_TIMEOUT_DEFAULT);
     this.fallbackAllowed = conf.getBoolean(CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
         CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
+    this.bindToWildCardAddress = conf
+        .getBoolean(CommonConfigurationKeys.IPC_CLIENT_BIND_WILDCARD_ADDR_KEY,
+            CommonConfigurationKeys.IPC_CLIENT_BIND_WILDCARD_ADDR_DEFAULT);
+    LOG.debug("{} set to true. Will bind client sockets to wildcard "
+            + "address.",
+        CommonConfigurationKeys.IPC_CLIENT_BIND_WILDCARD_ADDR_KEY);
+
     this.clientId = ClientId.getClientId();
     this.sendParamsExecutor = clientExcecutorFactory.refAndGetInstance();
     this.maxAsyncCalls = conf.getInt(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f42dafc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
index e16c2a3..0f9cfc3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
@@ -952,4 +952,20 @@ public class NetUtils {
     }
     return port;
   }
+
+  /**
+   * Return an @{@link InetAddress} to bind to. If bindWildCardAddress is true
+   * than returns null.
+   *
+   * @param localAddr
+   * @param bindWildCardAddress
+   * @returns InetAddress
+   */
+  public static InetAddress bindToLocalAddress(InetAddress localAddr, boolean
+      bindWildCardAddress) {
+    if (!bindWildCardAddress) {
+      return localAddr;
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f42dafc/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index fd72618..3a00131 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -3051,4 +3051,12 @@
       System tags to group related properties together.
     </description>
   </property>
+
+  <property>
+    <name>ipc.client.bind.wildcard.addr</name>
+    <value>false</value>
+    <description>When set to true Clients will bind socket to wildcard
+      address. (i.e 0.0.0.0)
+    </description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f42dafc/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
index b463c95..30176f2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
@@ -707,6 +707,14 @@ public class TestNetUtils {
     assertEquals(defaultAddr.trim(), NetUtils.getHostPortString(addr));
   }
 
+  @Test
+  public void testBindToLocalAddress() throws Exception {
+    assertNotNull(NetUtils
+        .bindToLocalAddress(NetUtils.getLocalInetAddress("127.0.0.1"), false));
+    assertNull(NetUtils
+        .bindToLocalAddress(NetUtils.getLocalInetAddress("127.0.0.1"), true));
+  }
+
   private <T> void assertBetterArrayEquals(T[] expect, T[]got) {
     String expectStr = StringUtils.join(expect, ", ");
     String gotStr = StringUtils.join(got, ", ");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/35] hadoop git commit: HDFS-13481. TestRollingFileSystemSinkWithHdfs#testFlushThread: test failed intermittently (Contributed by Gabor Bota via Daniel Templeton)

Posted by xy...@apache.org.
HDFS-13481. TestRollingFileSystemSinkWithHdfs#testFlushThread: test failed intermittently
(Contributed by Gabor Bota via Daniel Templeton)

Change-Id: I9921981dfa69669fe7912dd2a31ae8b638283204


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/87c23ef6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/87c23ef6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/87c23ef6

Branch: refs/heads/HDDS-4
Commit: 87c23ef643393c39e8353ca9f495b0c8f97cdbd9
Parents: f4d280f
Author: Daniel Templeton <te...@apache.org>
Authored: Wed May 2 16:54:42 2018 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Wed May 2 17:13:40 2018 -0700

----------------------------------------------------------------------
 .../hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java       | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/87c23ef6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java
index da85b9b..0f90d82 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSinkTestBase.java
@@ -182,7 +182,8 @@ public class RollingFileSystemSinkTestBase {
         .add(prefix + ".sink.mysink0.ignore-error", ignoreErrors)
         .add(prefix + ".sink.mysink0.allow-append", allowAppend)
         .add(prefix + ".sink.mysink0.roll-offset-interval-millis", 0)
-        .add(prefix + ".sink.mysink0.roll-interval", "1h");
+        .add(prefix + ".sink.mysink0.roll-interval", "1h")
+        .add("*.queue.capacity", 2);
 
     if (useSecureParams) {
       builder.add(prefix + ".sink.mysink0.keytab-key", SINK_KEYTAB_FILE_KEY)


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/35] hadoop git commit: YARN-8223. Improved yarn auxiliary service to load jar file from HDFS. Contributed by Zian Chen

Posted by xy...@apache.org.
YARN-8223.  Improved yarn auxiliary service to load jar file from HDFS.
            Contributed by Zian Chen


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8cdb032a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8cdb032a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8cdb032a

Branch: refs/heads/HDDS-4
Commit: 8cdb032aff4237d8d3970057d82290e4e32c4040
Parents: 6795f80
Author: Eric Yang <ey...@apache.org>
Authored: Fri May 4 12:36:31 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Fri May 4 12:36:31 2018 -0400

----------------------------------------------------------------------
 .../PluggableShuffleAndPluggableSort.md         | 44 ++++++++++++++++++++
 .../containermanager/AuxServices.java           | 19 ++++++++-
 2 files changed, 61 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdb032a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/PluggableShuffleAndPluggableSort.md
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/PluggableShuffleAndPluggableSort.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/PluggableShuffleAndPluggableSort.md
index 5ea0567..9e24103 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/PluggableShuffleAndPluggableSort.md
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/PluggableShuffleAndPluggableSort.md
@@ -67,6 +67,50 @@ The collector class configuration may specify a comma-separated list of collecto
 |:---- |:---- |:---- |
 | `yarn.nodemanager.aux-services` | `...,mapreduce_shuffle` | The auxiliary service name |
 | `yarn.nodemanager.aux-services.mapreduce_shuffle.class` | `org.apache.hadoop.mapred.ShuffleHandler` | The auxiliary service class to use |
+| `yarn.nodemanager.aux-services.%s.classpath` | NONE | local directory which includes the related jar file as well as all the dependencies’ jar file. We could specify the single jar file or use /dep/* to load all jars under the dep directory. |
+| `yarn.nodemanager.aux-services.%s.remote-classpath` | NONE | The remote absolute or relative path to jar file |
+
+#### Example of loading jar file from HDFS:
+
+```xml
+<configuration>
+    <property>
+        <name>yarn.nodemanager.aux-services</name>
+        <value>mapreduce_shuffle,AuxServiceFromHDFS</value>
+    </property>
+
+    <property>
+        <name>yarn.nodemanager.aux-services.AuxServiceFromHDFS.remote-classpath</name>
+        <value>/aux/test/aux-service-hdfs.jar</value>
+    </property>
+
+    <property>
+        <name>yarn.nodemanager.aux-services.AuxServiceFromHDFS.class&lt;/name>
+        <value>org.apache.auxtest.AuxServiceFromHDFS2</value>
+    </property>
+</configuration>
+```
+
+#### Example of loading jar file from local file system:
+
+```xml
+<configuration>
+    <property>
+        <name>yarn.nodemanager.aux-services</name>
+        <value>mapreduce_shuffle,AuxServiceFromHDFS</value>
+    </property>
+
+    <property>
+        <name>yarn.nodemanager.aux-services.AuxServiceFromHDFS.classpath</name>
+        <value>/aux/test/aux-service-hdfs.jar</value>
+    </property>
+
+    <property>
+        <name>yarn.nodemanager.aux-services.AuxServiceFromHDFS.class&lt;/name>
+        <value>org.apache.auxtest.AuxServiceFromHDFS2</value>
+    </property>
+</configuration>
+```
 
 **IMPORTANT:** If setting an auxiliary service in addition the default
 `mapreduce_shuffle` service, then a new service key should be added to the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdb032a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
index c8b7a76..3fe3cfd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
@@ -230,15 +230,30 @@ public class AuxServices extends AbstractService
               }
             }
             if (reDownload) {
+              LocalResourceType srcType = null;
+              String lowerDst = StringUtils.toLowerCase(src.toString());
+              if (lowerDst.endsWith(".jar")) {
+                srcType = LocalResourceType.FILE;
+              } else if (lowerDst.endsWith(".zip") ||
+                  lowerDst.endsWith(".tar.gz") || lowerDst.endsWith(".tgz")
+                  || lowerDst.endsWith(".tar")) {
+                srcType = LocalResourceType.ARCHIVE;
+              } else {
+                throw new YarnRuntimeException(
+                    "Can not unpack file from remote-file-path:" + src
+                        + "for aux-service:" + ".\n");
+              }
               LocalResource scRsrc = LocalResource.newInstance(
                   URL.fromURI(src.toUri()),
-                  LocalResourceType.ARCHIVE, LocalResourceVisibility.PRIVATE,
+                  srcType, LocalResourceVisibility.PRIVATE,
                   scFileStatus.getLen(), scFileStatus.getModificationTime());
               FSDownload download = new FSDownload(localLFS, null, conf,
                   downloadDest, scRsrc, null);
               try {
                 Path downloaded = download.call();
-                dest = new Path(downloaded + Path.SEPARATOR + "*");
+                // don't need to convert downloaded path into a dir
+                // since its already a jar path.
+                dest = downloaded;
               } catch (Exception ex) {
                 throw new YarnRuntimeException(
                     "Exception happend while downloading files "


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/35] hadoop git commit: YARN-8025. UsersManangers#getComputedResourceLimitForActiveUsers throws NPE due to preComputedActiveUserLimit is empty. Contributed by Tao Yang.

Posted by xy...@apache.org.
YARN-8025. UsersManangers#getComputedResourceLimitForActiveUsers throws NPE due to preComputedActiveUserLimit is empty. Contributed by Tao Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/67f239c4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/67f239c4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/67f239c4

Branch: refs/heads/HDDS-4
Commit: 67f239c42f676237290d18ddbbc9aec369267692
Parents: e9159db
Author: Weiwei Yang <ww...@apache.org>
Authored: Mon May 7 11:33:14 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Mon May 7 11:33:14 2018 +0800

----------------------------------------------------------------------
 .../resourcemanager/scheduler/capacity/UsersManager.java    | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/67f239c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
index 7287c5b..747a488 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
@@ -490,11 +490,12 @@ public class UsersManager implements AbstractUsersManager {
       Resource clusterResource, String nodePartition,
       SchedulingMode schedulingMode) {
 
-    Map<SchedulingMode, Resource> userLimitPerSchedulingMode = preComputedActiveUserLimit
-        .get(nodePartition);
+    Map<SchedulingMode, Resource> userLimitPerSchedulingMode;
 
     try {
       writeLock.lock();
+      userLimitPerSchedulingMode =
+          preComputedActiveUserLimit.get(nodePartition);
       if (isRecomputeNeeded(schedulingMode, nodePartition, true)) {
         // recompute
         userLimitPerSchedulingMode = reComputeUserLimits(userName,
@@ -548,11 +549,11 @@ public class UsersManager implements AbstractUsersManager {
       Resource clusterResource, String nodePartition,
       SchedulingMode schedulingMode) {
 
-    Map<SchedulingMode, Resource> userLimitPerSchedulingMode = preComputedAllUserLimit
-        .get(nodePartition);
+    Map<SchedulingMode, Resource> userLimitPerSchedulingMode;
 
     try {
       writeLock.lock();
+      userLimitPerSchedulingMode = preComputedAllUserLimit.get(nodePartition);
       if (isRecomputeNeeded(schedulingMode, nodePartition, false)) {
         // recompute
         userLimitPerSchedulingMode = reComputeUserLimits(userName,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org