Forráskód Böngészése

YARN-3978. Configurably turn off the saving of container info in Generic AHS (Eric Payne via jeagles)

(cherry picked from commit 3cd02b95224e9d43fd63a4ef9ac5c44f113f710d)
Jonathan Eagles 9 éve
szülő
commit
899df5bce0
10 módosított fájl, 146 hozzáadás és 12 törlés
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  3. 7 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
  4. 39 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
  5. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  6. 77 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
  7. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
  8. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
  9. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
  10. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -681,6 +681,9 @@ Release 2.7.2 - UNRELEASED
     YARN-3967. Fetch the application report from the AHS if the RM does not know about it.
     (Mit Desai via xgong)
 
+    YARN-3978. Configurably turn off the saving of container info in Generic AHS
+    (Eric Payne via jeagles)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -1435,6 +1435,15 @@ public class YarnConfiguration extends Configuration {
   public static final String APPLICATION_HISTORY_STORE =
       APPLICATION_HISTORY_PREFIX + "store-class";
 
+  /** Save container meta-info in the application history store. */
+  @Private
+  public static final String
+      APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO =
+        APPLICATION_HISTORY_PREFIX + "save-non-am-container-meta-info";
+  @Private
+  public static final boolean
+            DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO = true;
+
   /** URI for FileSystemApplicationHistoryStore */
   @Private
   public static final String FS_APPLICATION_HISTORY_STORE_URI =

+ 7 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java

@@ -266,11 +266,13 @@ public class AppBlock extends HtmlBlock {
             @Override
             public ContainerReport run() throws Exception {
               ContainerReport report = null;
-              try {
-                report = appBaseProt.getContainerReport(request)
-                    .getContainerReport();
-              } catch (ContainerNotFoundException ex) {
-                LOG.warn(ex.getMessage());
+              if (request.getContainerId() != null) {
+                  try {
+                    report = appBaseProt.getContainerReport(request)
+                        .getContainerReport();
+                  } catch (ContainerNotFoundException ex) {
+                    LOG.warn(ex.getMessage());
+                  }
               }
               return report;
             }

+ 39 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -173,6 +174,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
         .currentTimeMillis(), "");
   }
 
+  private boolean saveNonAMContainerMetaInfo;
+
   public RMContainerImpl(Container container,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext, String nodeLabelExpression) {
@@ -201,9 +204,21 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     this.readLock = lock.readLock();
     this.writeLock = lock.writeLock();
 
+    saveNonAMContainerMetaInfo = rmContext.getYarnConfiguration().getBoolean(
+       YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
+       YarnConfiguration
+                 .DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO);
+
     rmContext.getRMApplicationHistoryWriter().containerStarted(this);
-    rmContext.getSystemMetricsPublisher().containerCreated(
-        this, this.creationTime);
+
+    // If saveNonAMContainerMetaInfo is true, store system metrics for all
+    // containers. If false, and if this container is marked as the AM, metrics
+    // will still be published for this container, but that calculation happens
+    // later.
+    if (saveNonAMContainerMetaInfo) {
+      rmContext.getSystemMetricsPublisher().containerCreated(
+          this, this.creationTime);
+    }
   }
 
   @Override
@@ -376,6 +391,15 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     } finally {
       writeLock.unlock();
     }
+
+    // Even if saveNonAMContainerMetaInfo is not true, the AM container's system
+    // metrics still need to be saved so that the AM's logs can be accessed.
+    // This call to getSystemMetricsPublisher().containerCreated() is mutually
+    // exclusive with the one in the RMContainerImpl constructor.
+    if (!saveNonAMContainerMetaInfo && this.isAMContainer) {
+      rmContext.getSystemMetricsPublisher().containerCreated(
+          this, this.creationTime);
+    }
   }
   
   @Override
@@ -516,8 +540,19 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
 
       container.rmContext.getRMApplicationHistoryWriter().containerFinished(
         container);
-      container.rmContext.getSystemMetricsPublisher().containerFinished(
-          container, container.finishTime);
+
+      boolean saveNonAMContainerMetaInfo =
+          container.rmContext.getYarnConfiguration().getBoolean(
+              YarnConfiguration
+                .APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
+              YarnConfiguration
+                .DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO);
+
+      if (saveNonAMContainerMetaInfo || container.isAMContainer()) {
+        container.rmContext.getSystemMetricsPublisher().containerFinished(
+            container, container.finishTime);
+      }
+
     }
 
     private static void updateAttemptMetrics(RMContainerImpl container) {

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -1172,6 +1172,7 @@ public class TestClientRMService {
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
     ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
         yarnScheduler);
     when(rmContext.getRMApps()).thenReturn(apps);

+ 77 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java

@@ -19,12 +19,14 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
 
 import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.anyLong;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -270,4 +272,77 @@ public class TestRMContainerImpl {
     Assert.assertNull(scheduler.getRMContainer(containerId2)
         .getResourceRequests());
   }
+
+  @Test (timeout = 180000)
+  public void testStoreAllContainerMetrics() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
+    MockRM rm1 = new MockRM(conf);
+
+    SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
+    rm1.getRMContext().setSystemMetricsPublisher(publisher);
+
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("unknownhost:1234", 8000);
+    RMApp app1 = rm1.submitApp(1024);
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.RUNNING);
+
+    // request a container.
+    am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
+    ContainerId containerId2 = ContainerId.newContainerId(
+        am1.getApplicationAttemptId(), 2);
+    rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED);
+    am1.allocate(new ArrayList<ResourceRequest>(), new ArrayList<ContainerId>())
+        .getAllocatedContainers();
+    rm1.waitForState(nm1, containerId2, RMContainerState.ACQUIRED);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.COMPLETE);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
+    rm1.waitForState(nm1, containerId2, RMContainerState.COMPLETED);
+    rm1.stop();
+
+    // RMContainer should be publishing system metrics for all containers.
+    // Since there is 1 AM container and 1 non-AM container, there should be 2
+    // container created events and 2 container finished events.
+    verify(publisher, times(2)).containerCreated(any(RMContainer.class), anyLong());
+    verify(publisher, times(2)).containerFinished(any(RMContainer.class), anyLong());
+  }
+
+  @Test (timeout = 180000)
+  public void testStoreOnlyAMContainerMetrics() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
+    conf.setBoolean(
+        YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
+        false);
+    MockRM rm1 = new MockRM(conf);
+
+    SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
+    rm1.getRMContext().setSystemMetricsPublisher(publisher);
+
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("unknownhost:1234", 8000);
+    RMApp app1 = rm1.submitApp(1024);
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.RUNNING);
+
+    // request a container.
+    am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
+    ContainerId containerId2 = ContainerId.newContainerId(
+        am1.getApplicationAttemptId(), 2);
+    rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED);
+    am1.allocate(new ArrayList<ResourceRequest>(), new ArrayList<ContainerId>())
+        .getAllocatedContainers();
+    rm1.waitForState(nm1, containerId2, RMContainerState.ACQUIRED);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.COMPLETE);
+    nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
+    rm1.waitForState(nm1, containerId2, RMContainerState.COMPLETED);
+    rm1.stop();
+
+    // RMContainer should be publishing system metrics only for AM container.
+    verify(publisher, times(1)).containerCreated(any(RMContainer.class), anyLong());
+    verify(publisher, times(1)).containerFinished(any(RMContainer.class), anyLong());
+  }
 }

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java

@@ -270,6 +270,7 @@ public class TestChildQueueOrder {
     when(rmContext.getDispatcher()).thenReturn(drainDispatcher);
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
         app_0.getApplicationId(), 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java

@@ -168,6 +168,8 @@ public class TestLeafQueue {
     cs.start();
 
     when(spyRMContext.getScheduler()).thenReturn(cs);
+    when(spyRMContext.getYarnConfiguration())
+        .thenReturn(new YarnConfiguration());
     when(cs.getNumClusterNodes()).thenReturn(3);
   }
   

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java

@@ -129,7 +129,9 @@ public class TestReservations {
 
     spyRMContext = spy(rmContext);
     when(spyRMContext.getScheduler()).thenReturn(cs);
-    
+    when(spyRMContext.getYarnConfiguration())
+        .thenReturn(new YarnConfiguration());
+
     cs.setRMContext(spyRMContext);
     cs.init(csConf);
     cs.start();
@@ -642,6 +644,7 @@ public class TestReservations {
     when(rmContext.getDispatcher()).thenReturn(drainDispatcher);
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
         app_0.getApplicationId(), 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
@@ -711,6 +714,7 @@ public class TestReservations {
     when(rmContext.getDispatcher()).thenReturn(drainDispatcher);
     when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
     when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
+    when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
         app_0.getApplicationId(), 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

@@ -222,6 +222,7 @@ public class TestFifoScheduler {
         scheduler);
     ((RMContextImpl) rmContext).setSystemMetricsPublisher(
         mock(SystemMetricsPublisher.class));
+    ((RMContextImpl) rmContext).setYarnConfiguration(new YarnConfiguration());
 
     scheduler.setRMContext(rmContext);
     scheduler.init(conf);
@@ -303,6 +304,7 @@ public class TestFifoScheduler {
         scheduler);
     ((RMContextImpl) rmContext).setSystemMetricsPublisher(
         mock(SystemMetricsPublisher.class));
+    ((RMContextImpl) rmContext).setYarnConfiguration(new YarnConfiguration());
     NullRMNodeLabelsManager nlm = new NullRMNodeLabelsManager();
     nlm.init(new Configuration());
     rmContext.setNodeLabelManager(nlm);