|
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
@@ -38,7 +39,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.event.InlineDispatcher;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
|
|
-import org.apache.hadoop.yarn.server.api.records.LogAggregationStatus;
|
|
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
|
|
@@ -267,7 +267,7 @@ public class TestRMAppLogAggregationStatus {
|
|
|
|
|
|
// Finally, node1 finished its log aggregation and sent out its final
|
|
|
// log aggregation status. The log aggregation status for node1 should
|
|
|
- // be changed from TIME_OUT to Finished
|
|
|
+ // be changed from TIME_OUT to SUCCEEDED
|
|
|
Map<ApplicationId, LogAggregationReport> node1ReportForApp3 =
|
|
|
new HashMap<ApplicationId, LogAggregationReport>();
|
|
|
String messageForNode1_3 =
|
|
@@ -275,7 +275,7 @@ public class TestRMAppLogAggregationStatus {
|
|
|
+ System.currentTimeMillis();
|
|
|
LogAggregationReport report1_3 =
|
|
|
LogAggregationReport.newInstance(appId, nodeId1,
|
|
|
- LogAggregationStatus.FINISHED, messageForNode1_3);
|
|
|
+ LogAggregationStatus.SUCCEEDED, messageForNode1_3);
|
|
|
node1ReportForApp3.put(appId, report1_3);
|
|
|
node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
|
|
|
.newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
|
|
@@ -288,7 +288,7 @@ public class TestRMAppLogAggregationStatus {
|
|
|
for (Entry<NodeId, LogAggregationReport> report : logAggregationStatus
|
|
|
.entrySet()) {
|
|
|
if (report.getKey().equals(node1.getNodeID())) {
|
|
|
- Assert.assertEquals(LogAggregationStatus.FINISHED, report.getValue()
|
|
|
+ Assert.assertEquals(LogAggregationStatus.SUCCEEDED, report.getValue()
|
|
|
.getLogAggregationStatus());
|
|
|
Assert.assertEquals(messageForNode1_1 + messageForNode1_2
|
|
|
+ messageForNode1_3, report.getValue().getDiagnosticMessage());
|
|
@@ -303,6 +303,104 @@ public class TestRMAppLogAggregationStatus {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test (timeout = 10000)
|
|
|
+ public void testGetLogAggregationStatusForAppReport() {
|
|
|
+ YarnConfiguration conf = new YarnConfiguration();
|
|
|
+
|
|
|
+ // Disable the log aggregation
|
|
|
+ conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false);
|
|
|
+ RMAppImpl rmApp = (RMAppImpl)createRMApp(conf);
|
|
|
+ // The log aggregation status should be DISABLED.
|
|
|
+ Assert.assertEquals(LogAggregationStatus.DISABLED,
|
|
|
+ rmApp.getLogAggregationStatusForAppReport());
|
|
|
+
|
|
|
+ // Enable the log aggregation
|
|
|
+ conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
|
|
|
+ rmApp = (RMAppImpl)createRMApp(conf);
|
|
|
+ // If we do not know any NodeManagers for this application ,
|
|
|
+ // the log aggregation status will return null
|
|
|
+ Assert.assertNull(rmApp.getLogAggregationStatusForAppReport());
|
|
|
+
|
|
|
+ NodeId nodeId1 = NodeId.newInstance("localhost", 1111);
|
|
|
+ NodeId nodeId2 = NodeId.newInstance("localhost", 2222);
|
|
|
+ NodeId nodeId3 = NodeId.newInstance("localhost", 3333);
|
|
|
+ NodeId nodeId4 = NodeId.newInstance("localhost", 4444);
|
|
|
+
|
|
|
+ // If the log aggregation status for all NMs are NOT_START,
|
|
|
+ // the log aggregation status for this app will return NOT_START
|
|
|
+ rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
|
|
|
+ Assert.assertEquals(LogAggregationStatus.NOT_START,
|
|
|
+ rmApp.getLogAggregationStatusForAppReport());
|
|
|
+
|
|
|
+ rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.RUNNING, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.FAILED, ""));
|
|
|
+ Assert.assertEquals(LogAggregationStatus.RUNNING,
|
|
|
+ rmApp.getLogAggregationStatusForAppReport());
|
|
|
+
|
|
|
+ rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL));
|
|
|
+ Assert.assertTrue(RMAppImpl.isAppInFinalState(rmApp));
|
|
|
+
|
|
|
+ // If at least of one log aggregation status for one NM is TIME_OUT,
|
|
|
+ // others are SUCCEEDED, the log aggregation status for this app will
|
|
|
+ // return TIME_OUT
|
|
|
+ rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.TIME_OUT, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ Assert.assertEquals(LogAggregationStatus.TIME_OUT,
|
|
|
+ rmApp.getLogAggregationStatusForAppReport());
|
|
|
+
|
|
|
+ // If the log aggregation status for all NMs are SUCCEEDED and Application
|
|
|
+ // is at the final state, the log aggregation status for this app will
|
|
|
+ // return SUCCEEDED
|
|
|
+ rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ Assert.assertEquals(LogAggregationStatus.SUCCEEDED,
|
|
|
+ rmApp.getLogAggregationStatusForAppReport());
|
|
|
+
|
|
|
+ rmApp = (RMAppImpl)createRMApp(conf);
|
|
|
+ rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL));
|
|
|
+ Assert.assertTrue(RMAppImpl.isAppInFinalState(rmApp));
|
|
|
+ // If at least of one log aggregation status for one NM is FAILED,
|
|
|
+ // others are either SUCCEEDED or TIME_OUT, and this application is
|
|
|
+ // at the final state, the log aggregation status for this app
|
|
|
+ // will return FAILED
|
|
|
+ rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.TIME_OUT, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.FAILED, ""));
|
|
|
+ rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
|
|
|
+ rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
|
|
|
+ Assert.assertEquals(LogAggregationStatus.FAILED,
|
|
|
+ rmApp.getLogAggregationStatusForAppReport());
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
private RMApp createRMApp(Configuration conf) {
|
|
|
ApplicationSubmissionContext submissionContext =
|
|
|
ApplicationSubmissionContext.newInstance(appId, "test", "default",
|