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 jh...@apache.org on 2019/03/19 18:36:44 UTC

[hadoop] branch YARN-8200 updated (0bac160 -> a63d984)

This is an automated email from the ASF dual-hosted git repository.

jhung pushed a change to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


    from 0bac160  YARN-9397. Fix empty NMResourceInfo object test failures in branch-2
     new f6790f4  YARN-9291. Backport YARN-7637 to branch-2
     new 6f39788  YARN-7345. GPU Isolation: Incorrect minor device numbers written to devices.deny file. (Jonathan Hung via wangda)
     new 8880ec5  YARN-7143. FileNotFound handling in ResourceUtils is inconsistent
     new a4b7367  YARN-7383. Node resource is not parsed correctly for resource names containing dot. Contributed by Gergely Novák.
     new a63d984  YARN-8183. Fix ConcurrentModificationException inside RMAppAttemptMetrics#convertAtomicLongMaptoLongMap. (Suma Shivaprasad via wangda)

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../hadoop/yarn/util/resource/ResourceUtils.java   | 61 +++++++++-------------
 .../yarn/util/resource/TestResourceUtils.java      |  5 +-
 .../resources/resource-types/node-resources-2.xml  |  5 ++
 .../resources/resource-types/resource-types-4.xml  |  7 ++-
 .../recovery/NMNullStateStoreService.java          |  1 +
 .../impl/modules/gpu/gpu-module.c                  |  2 +-
 .../test/modules/gpu/test-gpu-module.cc            | 13 +++++
 .../resources/gpu/TestGpuResourceHandler.java      | 30 +++++++++++
 .../rmapp/attempt/RMAppAttemptMetrics.java         |  9 ++--
 9 files changed, 90 insertions(+), 43 deletions(-)


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


[hadoop] 02/05: YARN-7345. GPU Isolation: Incorrect minor device numbers written to devices.deny file. (Jonathan Hung via wangda)

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhung pushed a commit to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 6f39788d4b17cdcd2bfe9c4f4e4988ada4560f43
Author: Wangda Tan <wa...@apache.org>
AuthorDate: Thu Oct 19 14:45:44 2017 -0700

    YARN-7345. GPU Isolation: Incorrect minor device numbers written to devices.deny file. (Jonathan Hung via wangda)
    
    (cherry picked from commit c1b08ba720486e74461f0ec94a204c1ba4014c06)
    (cherry picked from commit 73cbc373e02f6d2c6c3cf173ff89939e527fcd11)
---
 .../native/container-executor/impl/modules/gpu/gpu-module.c |  2 +-
 .../container-executor/test/modules/gpu/test-gpu-module.cc  | 13 +++++++++++++
 2 files changed, 14 insertions(+), 1 deletion(-)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
index f96645d..1a1b164 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
@@ -108,7 +108,7 @@ static int internal_handle_gpu_request(
     char param_value[128];
     memset(param_value, 0, sizeof(param_value));
     snprintf(param_value, sizeof(param_value), "c %d:%d rwm",
-             major_device_number, i);
+             major_device_number, minor_devices[i]);
 
     int rc = update_cgroups_parameters_func_p("devices", "deny",
       container_id, param_value);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
index 7e41fb4..b3d93dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
@@ -165,6 +165,19 @@ TEST_F(TestGpuModule, test_verify_gpu_module_calls_cgroup_parameter) {
 
   // Verify cgroups parameters
   verify_param_updated_to_cgroups(0, NULL);
+
+  /* Test case 3: block 2 non-sequential devices */
+  cgroups_parameters_invoked.clear();
+  char* argv_2[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "1,3",
+                   (char*) "--container_id", container_id };
+  rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 5, argv_2);
+  ASSERT_EQ(0, rc) << "Should success.\n";
+
+  // Verify cgroups parameters
+  const char* expected_cgroups_argv_2[] = { "devices", "deny", container_id, "c 195:1 rwm",
+    "devices", "deny", container_id, "c 195:3 rwm"};
+  verify_param_updated_to_cgroups(8, expected_cgroups_argv_2);
 }
 
 TEST_F(TestGpuModule, test_illegal_cli_parameters) {


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


[hadoop] 05/05: YARN-8183. Fix ConcurrentModificationException inside RMAppAttemptMetrics#convertAtomicLongMaptoLongMap. (Suma Shivaprasad via wangda)

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhung pushed a commit to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit a63d984477cc0cf4e09fefa1bd88348e73bf824e
Author: Wangda Tan <wa...@apache.org>
AuthorDate: Tue Apr 24 17:42:17 2018 -0700

    YARN-8183. Fix ConcurrentModificationException inside RMAppAttemptMetrics#convertAtomicLongMaptoLongMap. (Suma Shivaprasad via wangda)
    
    Change-Id: I347871d672001653a3afe2e99adefd74e0d798cd
    (cherry picked from commit bb3c504764f807fccba7f28298a12e2296f284cb)
    (cherry picked from commit 3043a93d461fd8b9ccc2ff4b8d17e5430ed77615)
---
 .../resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java       | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java
index 0982ef9..e68c5d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -53,8 +54,8 @@ public class RMAppAttemptMetrics {
   
   private ReadLock readLock;
   private WriteLock writeLock;
-  private Map<String, AtomicLong> resourceUsageMap = new HashMap<>();
-  private Map<String, AtomicLong> preemptedResourceMap = new HashMap<>();
+  private Map<String, AtomicLong> resourceUsageMap = new ConcurrentHashMap<>();
+  private Map<String, AtomicLong> preemptedResourceMap = new ConcurrentHashMap<>();
   private RMContext rmContext;
 
   private int[][] localityStatistics =
@@ -97,7 +98,7 @@ public class RMAppAttemptMetrics {
   public Resource getResourcePreempted() {
     try {
       readLock.lock();
-      return resourcePreempted;
+      return Resource.newInstance(resourcePreempted);
     } finally {
       readLock.unlock();
     }
@@ -229,7 +230,7 @@ public class RMAppAttemptMetrics {
   }
 
   public Resource getApplicationAttemptHeadroom() {
-    return applicationHeadroom;
+    return Resource.newInstance(applicationHeadroom);
   }
 
   public void setApplicationAttemptHeadRoom(Resource headRoom) {


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


[hadoop] 04/05: YARN-7383. Node resource is not parsed correctly for resource names containing dot. Contributed by Gergely Novák.

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhung pushed a commit to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit a4b73679037606ec29386d42de6c32fffd1d9721
Author: Sunil G <su...@apache.org>
AuthorDate: Wed Dec 13 22:00:07 2017 +0530

    YARN-7383. Node resource is not parsed correctly for resource names containing dot. Contributed by Gergely Novák.
---
 .../apache/hadoop/yarn/util/resource/ResourceUtils.java   | 15 ++++++---------
 .../hadoop/yarn/util/resource/TestResourceUtils.java      |  5 ++++-
 .../test/resources/resource-types/node-resources-2.xml    |  5 +++++
 .../test/resources/resource-types/resource-types-4.xml    |  7 ++++++-
 4 files changed, 21 insertions(+), 11 deletions(-)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
index abf58a6..65eb5a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -461,21 +461,18 @@ public class ResourceUtils {
     for (Map.Entry<String, String> entry : conf) {
       String key = entry.getKey();
       String value = entry.getValue();
-
-      if (key.startsWith(YarnConfiguration.NM_RESOURCES_PREFIX)) {
-        addResourceInformation(key, value, nodeResources);
-      }
+      addResourceTypeInformation(key, value, nodeResources);
     }
 
     return nodeResources;
   }
 
-  private static void addResourceInformation(String prop, String value,
+  private static void addResourceTypeInformation(String prop, String value,
       Map<String, ResourceInformation> nodeResources) {
-    String[] parts = prop.split("\\.");
-    LOG.info("Found resource entry " + prop);
-    if (parts.length == 4) {
-      String resourceType = parts[3];
+    if (prop.startsWith(YarnConfiguration.NM_RESOURCES_PREFIX)) {
+      LOG.info("Found resource entry " + prop);
+      String resourceType = prop.substring(
+          YarnConfiguration.NM_RESOURCES_PREFIX.length());
       if (!nodeResources.containsKey(resourceType)) {
         nodeResources
             .put(resourceType, ResourceInformation.newInstance(resourceType));
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
index 80555ca..b511705 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
@@ -124,9 +124,10 @@ public class TestResourceUtils {
         new ResourceFileInformation("resource-types-3.xml", 3);
     testFile3.resourceNameUnitsMap.put("resource2", "");
     ResourceFileInformation testFile4 =
-        new ResourceFileInformation("resource-types-4.xml", 4);
+        new ResourceFileInformation("resource-types-4.xml", 5);
     testFile4.resourceNameUnitsMap.put("resource1", "G");
     testFile4.resourceNameUnitsMap.put("resource2", "m");
+    testFile4.resourceNameUnitsMap.put("yarn.io/gpu", "");
 
     ResourceFileInformation[] tests = {testFile1, testFile2, testFile3,
         testFile4};
@@ -292,6 +293,8 @@ public class TestResourceUtils {
         ResourceInformation.newInstance("resource1", "Gi", 5L));
     test3Resources.setResourceInformation("resource2",
         ResourceInformation.newInstance("resource2", "m", 2L));
+    test3Resources.setResourceInformation("yarn.io/gpu",
+        ResourceInformation.newInstance("yarn.io/gpu", "", 1));
     testRun.put("node-resources-2.xml", test3Resources);
 
     for (Map.Entry<String, Resource> entry : testRun.entrySet()) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-2.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-2.xml
index 9d9b3dc..382d5dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-2.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-2.xml
@@ -36,4 +36,9 @@ limitations under the License. See accompanying LICENSE file.
    <value>2m</value>
  </property>
 
+ <property>
+   <name>yarn.nodemanager.resource-type.yarn.io/gpu</name>
+   <value>1</value>
+ </property>
+
 </configuration>
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-4.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-4.xml
index c84316a..ea8d2bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-4.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-4.xml
@@ -18,7 +18,7 @@ limitations under the License. See accompanying LICENSE file.
 
  <property>
    <name>yarn.resource-types</name>
-   <value>resource1,resource2</value>
+   <value>resource1,resource2,yarn.io/gpu</value>
  </property>
 
  <property>
@@ -31,4 +31,9 @@ limitations under the License. See accompanying LICENSE file.
    <value>m</value>
  </property>
 
+ <property>
+   <name>yarn.resource-types.yarn.io/gpu.units</name>
+   <value></value>
+ </property>
+
 </configuration>


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


[hadoop] 01/05: YARN-9291. Backport YARN-7637 to branch-2

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhung pushed a commit to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit f6790f45eae1baea3f34e49a4bee77e6a7d750d5
Author: Jonathan Hung <jh...@linkedin.com>
AuthorDate: Mon Mar 18 17:59:54 2019 -0700

    YARN-9291. Backport YARN-7637 to branch-2
---
 .../recovery/NMNullStateStoreService.java          |  1 +
 .../resources/gpu/TestGpuResourceHandler.java      | 30 ++++++++++++++++++++++
 2 files changed, 31 insertions(+)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
index 7d1010f..95ec61a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
@@ -272,6 +272,7 @@ public class NMNullStateStoreService extends NMStateStoreService {
   public void storeAssignedResources(Container container,
       String resourceType, List<Serializable> assignedResources)
       throws IOException {
+    updateContainerResourceMapping(container, resourceType, assignedResources);
   }
 
   @Override
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/resources/gpu/TestGpuResourceHandler.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/resources/gpu/TestGpuResourceHandler.java
index b5796df..7a3bd02 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/resources/gpu/TestGpuResourceHandler.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/resources/gpu/TestGpuResourceHandler.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resource
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.gpu.GpuDevice;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.gpu.GpuDiscoverer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.util.resource.TestResourceUtils;
 import org.junit.Assert;
@@ -349,6 +350,35 @@ public class TestGpuResourceHandler {
   }
 
   @Test
+  public void testAllocationStoredWithNULLStateStore() throws Exception {
+    NMNullStateStoreService mockNMNULLStateStore = mock(NMNullStateStoreService.class);
+
+    Context nmnctx = mock(Context.class);
+    when(nmnctx.getNMStateStore()).thenReturn(mockNMNULLStateStore);
+
+    GpuResourceHandlerImpl gpuNULLStateResourceHandler =
+        new GpuResourceHandlerImpl(nmnctx, mockCGroupsHandler,
+        mockPrivilegedExecutor);
+
+    Configuration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.NM_GPU_ALLOWED_DEVICES, "0:0,1:1,2:3,3:4");
+    GpuDiscoverer.getInstance().initialize(conf);
+
+    gpuNULLStateResourceHandler.bootstrap(conf);
+    Assert.assertEquals(4,
+        gpuNULLStateResourceHandler.getGpuAllocator().getAvailableGpus());
+
+    /* Start container 1, asks 3 containers */
+    Container container = mockContainerWithGpuRequest(1, 3);
+    gpuNULLStateResourceHandler.preStart(container);
+
+    verify(nmnctx.getNMStateStore()).storeAssignedResources(container,
+        ResourceInformation.GPU_URI, Arrays
+            .<Serializable>asList(new GpuDevice(0, 0), new GpuDevice(1, 1),
+                new GpuDevice(2, 3)));
+  }
+
+  @Test
   public void testRecoverResourceAllocation() throws Exception {
     Configuration conf = new YarnConfiguration();
     conf.set(YarnConfiguration.NM_GPU_ALLOWED_DEVICES, "0:0,1:1,2:3,3:4");


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


[hadoop] 03/05: YARN-7143. FileNotFound handling in ResourceUtils is inconsistent

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhung pushed a commit to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 8880ec5480675a9a8e74b6fe4f9eababd2f53a5b
Author: Daniel Templeton <te...@apache.org>
AuthorDate: Thu Nov 9 10:36:49 2017 -0800

    YARN-7143. FileNotFound handling in ResourceUtils is inconsistent
    
    Change-Id: Ib1bb487e14a15edd2b5a42cf5078c5a2b295f069
    (cherry picked from commit db82a41d94872cea4d0c1bb1336916cebc2faeec)
---
 .../hadoop/yarn/util/resource/ResourceUtils.java   | 52 +++++++++-------------
 1 file changed, 22 insertions(+), 30 deletions(-)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
index f3edc74..abf58a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -338,18 +338,14 @@ public class ResourceUtils {
     if (!initializedResources) {
       synchronized (ResourceUtils.class) {
         if (!initializedResources) {
-          if (conf == null) {
-            conf = new YarnConfiguration();
-          }
-          try {
-            addResourcesFileToConf(resourceFile, conf);
-            LOG.debug("Found " + resourceFile + ", adding to configuration");
-          } catch (FileNotFoundException fe) {
-            LOG.info("Unable to find '" + resourceFile
-                + "'. Falling back to memory and vcores as resources.");
+          Configuration resConf = conf;
+
+          if (resConf == null) {
+            resConf = new YarnConfiguration();
           }
-          initializeResourcesMap(conf);
 
+          addResourcesFileToConf(resourceFile, resConf);
+          initializeResourcesMap(resConf);
         }
       }
     }
@@ -386,21 +382,17 @@ public class ResourceUtils {
   }
 
   private static void addResourcesFileToConf(String resourceFile,
-      Configuration conf) throws FileNotFoundException {
+      Configuration conf) {
     try {
       InputStream ris = getConfInputStream(resourceFile, conf);
       LOG.debug("Found " + resourceFile + ", adding to configuration");
       conf.addResource(ris);
     } catch (FileNotFoundException fe) {
-      throw fe;
-    } catch (IOException ie) {
+      LOG.info("Unable to find '" + resourceFile + "'.");
+    } catch (IOException | YarnException ex) {
       LOG.fatal("Exception trying to read resource types configuration '"
-          + resourceFile + "'.", ie);
-      throw new YarnRuntimeException(ie);
-    } catch (YarnException ye) {
-      LOG.fatal("YARN Exception trying to read resource types configuration '"
-          + resourceFile + "'.", ye);
-      throw new YarnRuntimeException(ye);
+          + resourceFile + "'.", ex);
+      throw new YarnRuntimeException(ex);
     }
   }
 
@@ -462,19 +454,19 @@ public class ResourceUtils {
   private static Map<String, ResourceInformation> initializeNodeResourceInformation(
       Configuration conf) {
     Map<String, ResourceInformation> nodeResources = new HashMap<>();
-    try {
-      addResourcesFileToConf(
-          YarnConfiguration.NODE_RESOURCES_CONFIGURATION_FILE, conf);
-      for (Map.Entry<String, String> entry : conf) {
-        String key = entry.getKey();
-        String value = entry.getValue();
-        if (key.startsWith(YarnConfiguration.NM_RESOURCES_PREFIX)) {
-          addResourceInformation(key, value, nodeResources);
-        }
+
+    addResourcesFileToConf(YarnConfiguration.NODE_RESOURCES_CONFIGURATION_FILE,
+        conf);
+
+    for (Map.Entry<String, String> entry : conf) {
+      String key = entry.getKey();
+      String value = entry.getValue();
+
+      if (key.startsWith(YarnConfiguration.NM_RESOURCES_PREFIX)) {
+        addResourceInformation(key, value, nodeResources);
       }
-    } catch (FileNotFoundException fe) {
-      LOG.info("Couldn't find node resources file");
     }
+
     return nodeResources;
   }
 


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