|
@@ -23,15 +23,12 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
@@ -47,6 +44,7 @@ import java.util.Random;
|
|
|
public class TestHadoopArchiveLogs {
|
|
|
|
|
|
private static final long CLUSTER_TIMESTAMP = System.currentTimeMillis();
|
|
|
+ private static final String USER = System.getProperty("user.name");
|
|
|
private static final int FILE_SIZE_INCREMENT = 4096;
|
|
|
private static final byte[] DUMMY_DATA = new byte[FILE_SIZE_INCREMENT];
|
|
|
static {
|
|
@@ -54,96 +52,117 @@ public class TestHadoopArchiveLogs {
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 10000)
|
|
|
- public void testCheckFiles() throws Exception {
|
|
|
+ public void testCheckFilesAndSeedApps() throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
HadoopArchiveLogs hal = new HadoopArchiveLogs(conf);
|
|
|
FileSystem fs = FileSystem.getLocal(conf);
|
|
|
Path rootLogDir = new Path("target", "logs");
|
|
|
String suffix = "logs";
|
|
|
- Path logDir = new Path(rootLogDir,
|
|
|
- new Path(System.getProperty("user.name"), suffix));
|
|
|
+ Path logDir = new Path(rootLogDir, new Path(USER, suffix));
|
|
|
fs.mkdirs(logDir);
|
|
|
|
|
|
- Assert.assertEquals(0, hal.eligibleApplications.size());
|
|
|
- ApplicationReport app1 = createAppReport(1); // no files found
|
|
|
- ApplicationReport app2 = createAppReport(2); // too few files
|
|
|
- Path app2Path = new Path(logDir, app2.getApplicationId().toString());
|
|
|
+ // no files found
|
|
|
+ ApplicationId appId1 = ApplicationId.newInstance(CLUSTER_TIMESTAMP, 1);
|
|
|
+ Path app1Path = new Path(logDir, appId1.toString());
|
|
|
+ fs.mkdirs(app1Path);
|
|
|
+ // too few files
|
|
|
+ ApplicationId appId2 = ApplicationId.newInstance(CLUSTER_TIMESTAMP, 2);
|
|
|
+ Path app2Path = new Path(logDir, appId2.toString());
|
|
|
fs.mkdirs(app2Path);
|
|
|
createFile(fs, new Path(app2Path, "file1"), 1);
|
|
|
hal.minNumLogFiles = 2;
|
|
|
- ApplicationReport app3 = createAppReport(3); // too large
|
|
|
- Path app3Path = new Path(logDir, app3.getApplicationId().toString());
|
|
|
+ // too large
|
|
|
+ ApplicationId appId3 = ApplicationId.newInstance(CLUSTER_TIMESTAMP, 3);
|
|
|
+ Path app3Path = new Path(logDir, appId3.toString());
|
|
|
fs.mkdirs(app3Path);
|
|
|
createFile(fs, new Path(app3Path, "file1"), 2);
|
|
|
createFile(fs, new Path(app3Path, "file2"), 5);
|
|
|
hal.maxTotalLogsSize = FILE_SIZE_INCREMENT * 6;
|
|
|
- ApplicationReport app4 = createAppReport(4); // has har already
|
|
|
- Path app4Path = new Path(logDir, app4.getApplicationId().toString());
|
|
|
+ // has har already
|
|
|
+ ApplicationId appId4 = ApplicationId.newInstance(CLUSTER_TIMESTAMP, 4);
|
|
|
+ Path app4Path = new Path(logDir, appId4.toString());
|
|
|
fs.mkdirs(app4Path);
|
|
|
- createFile(fs, new Path(app4Path, app4.getApplicationId() + ".har"), 1);
|
|
|
- ApplicationReport app5 = createAppReport(5); // just right
|
|
|
- Path app5Path = new Path(logDir, app5.getApplicationId().toString());
|
|
|
+ createFile(fs, new Path(app4Path, appId4 + ".har"), 1);
|
|
|
+ // just right
|
|
|
+ ApplicationId appId5 = ApplicationId.newInstance(CLUSTER_TIMESTAMP, 5);
|
|
|
+ Path app5Path = new Path(logDir, appId5.toString());
|
|
|
fs.mkdirs(app5Path);
|
|
|
createFile(fs, new Path(app5Path, "file1"), 2);
|
|
|
createFile(fs, new Path(app5Path, "file2"), 3);
|
|
|
- hal.eligibleApplications.add(app1);
|
|
|
- hal.eligibleApplications.add(app2);
|
|
|
- hal.eligibleApplications.add(app3);
|
|
|
- hal.eligibleApplications.add(app4);
|
|
|
- hal.eligibleApplications.add(app5);
|
|
|
|
|
|
- hal.checkFiles(fs, rootLogDir, suffix);
|
|
|
+ Assert.assertEquals(0, hal.eligibleApplications.size());
|
|
|
+ hal.checkFilesAndSeedApps(fs, rootLogDir, suffix);
|
|
|
Assert.assertEquals(1, hal.eligibleApplications.size());
|
|
|
- Assert.assertEquals(app5, hal.eligibleApplications.iterator().next());
|
|
|
+ Assert.assertEquals(appId5.toString(),
|
|
|
+ hal.eligibleApplications.iterator().next().getAppId());
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 10000)
|
|
|
public void testCheckMaxEligible() throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
- HadoopArchiveLogs hal = new HadoopArchiveLogs(conf);
|
|
|
- ApplicationReport app1 = createAppReport(1);
|
|
|
+ HadoopArchiveLogs.AppInfo app1 = new HadoopArchiveLogs.AppInfo(
|
|
|
+ ApplicationId.newInstance(CLUSTER_TIMESTAMP, 1).toString(), USER);
|
|
|
app1.setFinishTime(CLUSTER_TIMESTAMP - 5);
|
|
|
- ApplicationReport app2 = createAppReport(2);
|
|
|
+ HadoopArchiveLogs.AppInfo app2 = new HadoopArchiveLogs.AppInfo(
|
|
|
+ ApplicationId.newInstance(CLUSTER_TIMESTAMP, 2).toString(), USER);
|
|
|
app2.setFinishTime(CLUSTER_TIMESTAMP - 10);
|
|
|
- ApplicationReport app3 = createAppReport(3);
|
|
|
- app3.setFinishTime(CLUSTER_TIMESTAMP + 5);
|
|
|
- ApplicationReport app4 = createAppReport(4);
|
|
|
- app4.setFinishTime(CLUSTER_TIMESTAMP + 10);
|
|
|
- ApplicationReport app5 = createAppReport(5);
|
|
|
- app5.setFinishTime(CLUSTER_TIMESTAMP);
|
|
|
+ HadoopArchiveLogs.AppInfo app3 = new HadoopArchiveLogs.AppInfo(
|
|
|
+ ApplicationId.newInstance(CLUSTER_TIMESTAMP, 3).toString(), USER);
|
|
|
+ // app3 has no finish time set
|
|
|
+ HadoopArchiveLogs.AppInfo app4 = new HadoopArchiveLogs.AppInfo(
|
|
|
+ ApplicationId.newInstance(CLUSTER_TIMESTAMP, 4).toString(), USER);
|
|
|
+ app4.setFinishTime(CLUSTER_TIMESTAMP + 5);
|
|
|
+ HadoopArchiveLogs.AppInfo app5 = new HadoopArchiveLogs.AppInfo(
|
|
|
+ ApplicationId.newInstance(CLUSTER_TIMESTAMP, 5).toString(), USER);
|
|
|
+ app5.setFinishTime(CLUSTER_TIMESTAMP + 10);
|
|
|
+ HadoopArchiveLogs.AppInfo app6 = new HadoopArchiveLogs.AppInfo(
|
|
|
+ ApplicationId.newInstance(CLUSTER_TIMESTAMP, 6).toString(), USER);
|
|
|
+ // app6 has no finish time set
|
|
|
+ HadoopArchiveLogs.AppInfo app7 = new HadoopArchiveLogs.AppInfo(
|
|
|
+ ApplicationId.newInstance(CLUSTER_TIMESTAMP, 7).toString(), USER);
|
|
|
+ app7.setFinishTime(CLUSTER_TIMESTAMP);
|
|
|
+ HadoopArchiveLogs hal = new HadoopArchiveLogs(conf);
|
|
|
Assert.assertEquals(0, hal.eligibleApplications.size());
|
|
|
hal.eligibleApplications.add(app1);
|
|
|
hal.eligibleApplications.add(app2);
|
|
|
hal.eligibleApplications.add(app3);
|
|
|
hal.eligibleApplications.add(app4);
|
|
|
hal.eligibleApplications.add(app5);
|
|
|
+ hal.eligibleApplications.add(app6);
|
|
|
+ hal.eligibleApplications.add(app7);
|
|
|
+ Assert.assertEquals(7, hal.eligibleApplications.size());
|
|
|
hal.maxEligible = -1;
|
|
|
hal.checkMaxEligible();
|
|
|
+ Assert.assertEquals(7, hal.eligibleApplications.size());
|
|
|
+ hal.maxEligible = 6;
|
|
|
+ hal.checkMaxEligible();
|
|
|
+ Assert.assertEquals(6, hal.eligibleApplications.size());
|
|
|
+ Assert.assertFalse(hal.eligibleApplications.contains(app5));
|
|
|
+ hal.maxEligible = 5;
|
|
|
+ hal.checkMaxEligible();
|
|
|
Assert.assertEquals(5, hal.eligibleApplications.size());
|
|
|
-
|
|
|
+ Assert.assertFalse(hal.eligibleApplications.contains(app4));
|
|
|
hal.maxEligible = 4;
|
|
|
hal.checkMaxEligible();
|
|
|
Assert.assertEquals(4, hal.eligibleApplications.size());
|
|
|
- Assert.assertFalse(hal.eligibleApplications.contains(app4));
|
|
|
-
|
|
|
+ Assert.assertFalse(hal.eligibleApplications.contains(app7));
|
|
|
hal.maxEligible = 3;
|
|
|
hal.checkMaxEligible();
|
|
|
Assert.assertEquals(3, hal.eligibleApplications.size());
|
|
|
- Assert.assertFalse(hal.eligibleApplications.contains(app3));
|
|
|
-
|
|
|
+ Assert.assertFalse(hal.eligibleApplications.contains(app1));
|
|
|
hal.maxEligible = 2;
|
|
|
hal.checkMaxEligible();
|
|
|
Assert.assertEquals(2, hal.eligibleApplications.size());
|
|
|
- Assert.assertFalse(hal.eligibleApplications.contains(app5));
|
|
|
-
|
|
|
+ Assert.assertFalse(hal.eligibleApplications.contains(app2));
|
|
|
hal.maxEligible = 1;
|
|
|
hal.checkMaxEligible();
|
|
|
Assert.assertEquals(1, hal.eligibleApplications.size());
|
|
|
- Assert.assertFalse(hal.eligibleApplications.contains(app1));
|
|
|
+ Assert.assertFalse(hal.eligibleApplications.contains(app6));
|
|
|
+ Assert.assertTrue(hal.eligibleApplications.contains(app3));
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 10000)
|
|
|
- public void testFindAggregatedApps() throws Exception {
|
|
|
+ public void testFilterAppsByAggregatedStatus() throws Exception {
|
|
|
MiniYARNCluster yarnCluster = null;
|
|
|
try {
|
|
|
Configuration conf = new Configuration();
|
|
@@ -156,32 +175,66 @@ public class TestHadoopArchiveLogs {
|
|
|
conf = yarnCluster.getConfig();
|
|
|
|
|
|
RMContext rmContext = yarnCluster.getResourceManager().getRMContext();
|
|
|
- RMAppImpl app1 = (RMAppImpl)createRMApp(1, conf, rmContext,
|
|
|
+ RMAppImpl appImpl1 = (RMAppImpl)createRMApp(1, conf, rmContext,
|
|
|
LogAggregationStatus.DISABLED);
|
|
|
- RMAppImpl app2 = (RMAppImpl)createRMApp(2, conf, rmContext,
|
|
|
+ RMAppImpl appImpl2 = (RMAppImpl)createRMApp(2, conf, rmContext,
|
|
|
LogAggregationStatus.FAILED);
|
|
|
- RMAppImpl app3 = (RMAppImpl)createRMApp(3, conf, rmContext,
|
|
|
+ RMAppImpl appImpl3 = (RMAppImpl)createRMApp(3, conf, rmContext,
|
|
|
LogAggregationStatus.NOT_START);
|
|
|
- RMAppImpl app4 = (RMAppImpl)createRMApp(4, conf, rmContext,
|
|
|
+ RMAppImpl appImpl4 = (RMAppImpl)createRMApp(4, conf, rmContext,
|
|
|
LogAggregationStatus.SUCCEEDED);
|
|
|
- RMAppImpl app5 = (RMAppImpl)createRMApp(5, conf, rmContext,
|
|
|
+ RMAppImpl appImpl5 = (RMAppImpl)createRMApp(5, conf, rmContext,
|
|
|
LogAggregationStatus.RUNNING);
|
|
|
- RMAppImpl app6 = (RMAppImpl)createRMApp(6, conf, rmContext,
|
|
|
+ RMAppImpl appImpl6 = (RMAppImpl)createRMApp(6, conf, rmContext,
|
|
|
LogAggregationStatus.RUNNING_WITH_FAILURE);
|
|
|
- RMAppImpl app7 = (RMAppImpl)createRMApp(7, conf, rmContext,
|
|
|
+ RMAppImpl appImpl7 = (RMAppImpl)createRMApp(7, conf, rmContext,
|
|
|
LogAggregationStatus.TIME_OUT);
|
|
|
- rmContext.getRMApps().put(app1.getApplicationId(), app1);
|
|
|
- rmContext.getRMApps().put(app2.getApplicationId(), app2);
|
|
|
- rmContext.getRMApps().put(app3.getApplicationId(), app3);
|
|
|
- rmContext.getRMApps().put(app4.getApplicationId(), app4);
|
|
|
- rmContext.getRMApps().put(app5.getApplicationId(), app5);
|
|
|
- rmContext.getRMApps().put(app6.getApplicationId(), app6);
|
|
|
- rmContext.getRMApps().put(app7.getApplicationId(), app7);
|
|
|
+ RMAppImpl appImpl8 = (RMAppImpl)createRMApp(8, conf, rmContext,
|
|
|
+ LogAggregationStatus.SUCCEEDED);
|
|
|
+ rmContext.getRMApps().put(appImpl1.getApplicationId(), appImpl1);
|
|
|
+ rmContext.getRMApps().put(appImpl2.getApplicationId(), appImpl2);
|
|
|
+ rmContext.getRMApps().put(appImpl3.getApplicationId(), appImpl3);
|
|
|
+ rmContext.getRMApps().put(appImpl4.getApplicationId(), appImpl4);
|
|
|
+ rmContext.getRMApps().put(appImpl5.getApplicationId(), appImpl5);
|
|
|
+ rmContext.getRMApps().put(appImpl6.getApplicationId(), appImpl6);
|
|
|
+ rmContext.getRMApps().put(appImpl7.getApplicationId(), appImpl7);
|
|
|
+ // appImpl8 is not in the RM
|
|
|
|
|
|
HadoopArchiveLogs hal = new HadoopArchiveLogs(conf);
|
|
|
Assert.assertEquals(0, hal.eligibleApplications.size());
|
|
|
- hal.findAggregatedApps();
|
|
|
- Assert.assertEquals(2, hal.eligibleApplications.size());
|
|
|
+ hal.eligibleApplications.add(
|
|
|
+ new HadoopArchiveLogs.AppInfo(appImpl1.getApplicationId().toString(),
|
|
|
+ USER));
|
|
|
+ hal.eligibleApplications.add(
|
|
|
+ new HadoopArchiveLogs.AppInfo(appImpl2.getApplicationId().toString(),
|
|
|
+ USER));
|
|
|
+ hal.eligibleApplications.add(
|
|
|
+ new HadoopArchiveLogs.AppInfo(appImpl3.getApplicationId().toString(),
|
|
|
+ USER));
|
|
|
+ HadoopArchiveLogs.AppInfo app4 =
|
|
|
+ new HadoopArchiveLogs.AppInfo(appImpl4.getApplicationId().toString(),
|
|
|
+ USER);
|
|
|
+ hal.eligibleApplications.add(app4);
|
|
|
+ hal.eligibleApplications.add(
|
|
|
+ new HadoopArchiveLogs.AppInfo(appImpl5.getApplicationId().toString(),
|
|
|
+ USER));
|
|
|
+ hal.eligibleApplications.add(
|
|
|
+ new HadoopArchiveLogs.AppInfo(appImpl6.getApplicationId().toString(),
|
|
|
+ USER));
|
|
|
+ HadoopArchiveLogs.AppInfo app7 =
|
|
|
+ new HadoopArchiveLogs.AppInfo(appImpl7.getApplicationId().toString(),
|
|
|
+ USER);
|
|
|
+ hal.eligibleApplications.add(app7);
|
|
|
+ HadoopArchiveLogs.AppInfo app8 =
|
|
|
+ new HadoopArchiveLogs.AppInfo(appImpl8.getApplicationId().toString(),
|
|
|
+ USER);
|
|
|
+ hal.eligibleApplications.add(app8);
|
|
|
+ Assert.assertEquals(8, hal.eligibleApplications.size());
|
|
|
+ hal.filterAppsByAggregatedStatus();
|
|
|
+ Assert.assertEquals(3, hal.eligibleApplications.size());
|
|
|
+ Assert.assertTrue(hal.eligibleApplications.contains(app4));
|
|
|
+ Assert.assertTrue(hal.eligibleApplications.contains(app7));
|
|
|
+ Assert.assertTrue(hal.eligibleApplications.contains(app8));
|
|
|
} finally {
|
|
|
if (yarnCluster != null) {
|
|
|
yarnCluster.stop();
|
|
@@ -193,10 +246,12 @@ public class TestHadoopArchiveLogs {
|
|
|
public void testGenerateScript() throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
HadoopArchiveLogs hal = new HadoopArchiveLogs(conf);
|
|
|
- ApplicationReport app1 = createAppReport(1);
|
|
|
- ApplicationReport app2 = createAppReport(2);
|
|
|
- hal.eligibleApplications.add(app1);
|
|
|
- hal.eligibleApplications.add(app2);
|
|
|
+ ApplicationId app1 = ApplicationId.newInstance(CLUSTER_TIMESTAMP, 1);
|
|
|
+ ApplicationId app2 = ApplicationId.newInstance(CLUSTER_TIMESTAMP, 2);
|
|
|
+ hal.eligibleApplications.add(new HadoopArchiveLogs.AppInfo(app1.toString(),
|
|
|
+ USER));
|
|
|
+ hal.eligibleApplications.add(new HadoopArchiveLogs.AppInfo(app2.toString(),
|
|
|
+ USER));
|
|
|
|
|
|
File localScript = new File("target", "script.sh");
|
|
|
Path workingDir = new Path("/tmp", "working");
|
|
@@ -213,22 +268,16 @@ public class TestHadoopArchiveLogs {
|
|
|
Assert.assertEquals("set -e", lines[1]);
|
|
|
Assert.assertEquals("set -x", lines[2]);
|
|
|
Assert.assertEquals("if [ \"$YARN_SHELL_ID\" == \"1\" ]; then", lines[3]);
|
|
|
- if (lines[4].contains(app1.getApplicationId().toString())) {
|
|
|
- Assert.assertEquals("\tappId=\"" + app1.getApplicationId().toString()
|
|
|
- + "\"", lines[4]);
|
|
|
- Assert.assertEquals("\tappId=\"" + app2.getApplicationId().toString()
|
|
|
- + "\"", lines[7]);
|
|
|
+ if (lines[4].contains(app1.toString())) {
|
|
|
+ Assert.assertEquals("\tappId=\"" + app1.toString() + "\"", lines[4]);
|
|
|
+ Assert.assertEquals("\tappId=\"" + app2.toString() + "\"", lines[7]);
|
|
|
} else {
|
|
|
- Assert.assertEquals("\tappId=\"" + app2.getApplicationId().toString()
|
|
|
- + "\"", lines[4]);
|
|
|
- Assert.assertEquals("\tappId=\"" + app1.getApplicationId().toString()
|
|
|
- + "\"", lines[7]);
|
|
|
+ Assert.assertEquals("\tappId=\"" + app2.toString() + "\"", lines[4]);
|
|
|
+ Assert.assertEquals("\tappId=\"" + app1.toString() + "\"", lines[7]);
|
|
|
}
|
|
|
- Assert.assertEquals("\tuser=\"" + System.getProperty("user.name") + "\"",
|
|
|
- lines[5]);
|
|
|
+ Assert.assertEquals("\tuser=\"" + USER + "\"", lines[5]);
|
|
|
Assert.assertEquals("elif [ \"$YARN_SHELL_ID\" == \"2\" ]; then", lines[6]);
|
|
|
- Assert.assertEquals("\tuser=\"" + System.getProperty("user.name") + "\"",
|
|
|
- lines[8]);
|
|
|
+ Assert.assertEquals("\tuser=\"" + USER + "\"", lines[8]);
|
|
|
Assert.assertEquals("else", lines[9]);
|
|
|
Assert.assertEquals("\techo \"Unknown Mapping!\"", lines[10]);
|
|
|
Assert.assertEquals("\texit 1", lines[11]);
|
|
@@ -241,15 +290,23 @@ public class TestHadoopArchiveLogs {
|
|
|
remoteRootLogDir.toString() + " -suffix " + suffix, lines[15]);
|
|
|
}
|
|
|
|
|
|
- private static ApplicationReport createAppReport(int id) {
|
|
|
- ApplicationId appId = ApplicationId.newInstance(CLUSTER_TIMESTAMP, id);
|
|
|
- return ApplicationReport.newInstance(
|
|
|
- appId,
|
|
|
- ApplicationAttemptId.newInstance(appId, 1),
|
|
|
- System.getProperty("user.name"),
|
|
|
- null, null, null, 0, null, YarnApplicationState.FINISHED, null,
|
|
|
- null, 0L, 0L, FinalApplicationStatus.SUCCEEDED, null, null, 100f,
|
|
|
- null, null);
|
|
|
+ /**
|
|
|
+ * If this test failes, then a new Log Aggregation Status was added. Make
|
|
|
+ * sure that {@link HadoopArchiveLogs#filterAppsByAggregatedStatus()} and this test
|
|
|
+ * are updated as well, if necessary.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test(timeout = 5000)
|
|
|
+ public void testStatuses() throws Exception {
|
|
|
+ LogAggregationStatus[] statuses = new LogAggregationStatus[7];
|
|
|
+ statuses[0] = LogAggregationStatus.DISABLED;
|
|
|
+ statuses[1] = LogAggregationStatus.NOT_START;
|
|
|
+ statuses[2] = LogAggregationStatus.RUNNING;
|
|
|
+ statuses[3] = LogAggregationStatus.RUNNING_WITH_FAILURE;
|
|
|
+ statuses[4] = LogAggregationStatus.SUCCEEDED;
|
|
|
+ statuses[5] = LogAggregationStatus.FAILED;
|
|
|
+ statuses[6] = LogAggregationStatus.TIME_OUT;
|
|
|
+ Assert.assertArrayEquals(statuses, LogAggregationStatus.values());
|
|
|
}
|
|
|
|
|
|
private static void createFile(FileSystem fs, Path p, long sizeMultiple)
|
|
@@ -265,6 +322,7 @@ public class TestHadoopArchiveLogs {
|
|
|
out.close();
|
|
|
}
|
|
|
}
|
|
|
+ Assert.assertTrue(fs.exists(p));
|
|
|
}
|
|
|
|
|
|
private static RMApp createRMApp(int id, Configuration conf, RMContext rmContext,
|
|
@@ -272,11 +330,10 @@ public class TestHadoopArchiveLogs {
|
|
|
ApplicationId appId = ApplicationId.newInstance(CLUSTER_TIMESTAMP, id);
|
|
|
ApplicationSubmissionContext submissionContext =
|
|
|
ApplicationSubmissionContext.newInstance(appId, "test", "default",
|
|
|
- Priority.newInstance(0), null, false, true,
|
|
|
+ Priority.newInstance(0), null, true, true,
|
|
|
2, Resource.newInstance(10, 2), "test");
|
|
|
return new RMAppImpl(appId, rmContext, conf, "test",
|
|
|
- System.getProperty("user.name"), "default", submissionContext,
|
|
|
- rmContext.getScheduler(),
|
|
|
+ USER, "default", submissionContext, rmContext.getScheduler(),
|
|
|
rmContext.getApplicationMasterService(),
|
|
|
System.currentTimeMillis(), "test",
|
|
|
null, null) {
|