|
@@ -41,16 +41,11 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.MRApp;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.MockAppContext;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.MockJobs;
|
|
|
-import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.webapp.TestAMWebApp;
|
|
|
-import org.apache.hadoop.yarn.Clock;
|
|
|
-import org.apache.hadoop.yarn.ClusterInfo;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
-import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.webapp.log.AggregatedLogsPage;
|
|
|
import org.apache.hadoop.yarn.webapp.test.WebAppTests;
|
|
|
import org.junit.Test;
|
|
@@ -61,92 +56,17 @@ import com.google.inject.Injector;
|
|
|
public class TestHSWebApp {
|
|
|
private static final Log LOG = LogFactory.getLog(TestHSWebApp.class);
|
|
|
|
|
|
- static class TestAppContext implements AppContext {
|
|
|
- final ApplicationAttemptId appAttemptID;
|
|
|
- final ApplicationId appID;
|
|
|
- final String user = MockJobs.newUserName();
|
|
|
- final Map<JobId, Job> jobs;
|
|
|
- final long startTime = System.currentTimeMillis();
|
|
|
-
|
|
|
- TestAppContext(int appid, int numJobs, int numTasks, int numAttempts,
|
|
|
- boolean hasFailedTasks) {
|
|
|
- appID = MockJobs.newAppID(appid);
|
|
|
- appAttemptID = ApplicationAttemptId.newInstance(appID, 0);
|
|
|
- jobs = MockJobs.newJobs(appID, numJobs, numTasks, numAttempts,
|
|
|
- hasFailedTasks);
|
|
|
- }
|
|
|
-
|
|
|
- TestAppContext(int appid, int numJobs, int numTasks, int numAttempts) {
|
|
|
- this(appid, numJobs, numTasks, numAttempts, false);
|
|
|
- }
|
|
|
-
|
|
|
- TestAppContext() {
|
|
|
- this(0, 1, 1, 1);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public ApplicationAttemptId getApplicationAttemptId() {
|
|
|
- return appAttemptID;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public ApplicationId getApplicationID() {
|
|
|
- return appID;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public CharSequence getUser() {
|
|
|
- return user;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public Job getJob(JobId jobID) {
|
|
|
- return jobs.get(jobID);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public Map<JobId, Job> getAllJobs() {
|
|
|
- return jobs; // OK
|
|
|
- }
|
|
|
-
|
|
|
- @SuppressWarnings("rawtypes")
|
|
|
- @Override
|
|
|
- public EventHandler getEventHandler() {
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public Clock getClock() {
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public String getApplicationName() {
|
|
|
- return "TestApp";
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public long getStartTime() {
|
|
|
- return startTime;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public ClusterInfo getClusterInfo() {
|
|
|
- return null;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@Test public void testAppControllerIndex() {
|
|
|
- TestAppContext ctx = new TestAppContext();
|
|
|
+ MockAppContext ctx = new MockAppContext(0, 1, 1, 1);
|
|
|
Injector injector = WebAppTests.createMockInjector(AppContext.class, ctx);
|
|
|
HsController controller = injector.getInstance(HsController.class);
|
|
|
controller.index();
|
|
|
- assertEquals(ctx.appID.toString(), controller.get(APP_ID,""));
|
|
|
+ assertEquals(ctx.getApplicationID().toString(), controller.get(APP_ID,""));
|
|
|
}
|
|
|
|
|
|
@Test public void testJobView() {
|
|
|
LOG.info("HsJobPage");
|
|
|
- AppContext appContext = new TestAppContext();
|
|
|
+ AppContext appContext = new MockAppContext(0, 1, 1, 1);
|
|
|
Map<String, String> params = TestAMWebApp.getJobParams(appContext);
|
|
|
WebAppTests.testPage(HsJobPage.class, AppContext.class, appContext, params);
|
|
|
}
|
|
@@ -154,7 +74,7 @@ public class TestHSWebApp {
|
|
|
@Test
|
|
|
public void testTasksView() {
|
|
|
LOG.info("HsTasksPage");
|
|
|
- AppContext appContext = new TestAppContext();
|
|
|
+ AppContext appContext = new MockAppContext(0, 1, 1, 1);
|
|
|
Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
|
|
|
WebAppTests.testPage(HsTasksPage.class, AppContext.class, appContext,
|
|
|
params);
|
|
@@ -163,7 +83,7 @@ public class TestHSWebApp {
|
|
|
@Test
|
|
|
public void testTaskView() {
|
|
|
LOG.info("HsTaskPage");
|
|
|
- AppContext appContext = new TestAppContext();
|
|
|
+ AppContext appContext = new MockAppContext(0, 1, 1, 1);
|
|
|
Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
|
|
|
WebAppTests
|
|
|
.testPage(HsTaskPage.class, AppContext.class, appContext, params);
|
|
@@ -171,7 +91,7 @@ public class TestHSWebApp {
|
|
|
|
|
|
@Test public void testAttemptsWithJobView() {
|
|
|
LOG.info("HsAttemptsPage with data");
|
|
|
- TestAppContext ctx = new TestAppContext();
|
|
|
+ MockAppContext ctx = new MockAppContext(0, 1, 1, 1);
|
|
|
JobId id = ctx.getAllJobs().keySet().iterator().next();
|
|
|
Map<String, String> params = new HashMap<String,String>();
|
|
|
params.put(JOB_ID, id.toString());
|
|
@@ -183,7 +103,7 @@ public class TestHSWebApp {
|
|
|
|
|
|
@Test public void testAttemptsView() {
|
|
|
LOG.info("HsAttemptsPage");
|
|
|
- AppContext appContext = new TestAppContext();
|
|
|
+ AppContext appContext = new MockAppContext(0, 1, 1, 1);
|
|
|
Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
|
|
|
WebAppTests.testPage(HsAttemptsPage.class, AppContext.class,
|
|
|
appContext, params);
|
|
@@ -192,18 +112,18 @@ public class TestHSWebApp {
|
|
|
@Test public void testConfView() {
|
|
|
LOG.info("HsConfPage");
|
|
|
WebAppTests.testPage(HsConfPage.class, AppContext.class,
|
|
|
- new TestAppContext());
|
|
|
+ new MockAppContext(0, 1, 1, 1));
|
|
|
}
|
|
|
|
|
|
@Test public void testAboutView() {
|
|
|
LOG.info("HsAboutPage");
|
|
|
WebAppTests.testPage(HsAboutPage.class, AppContext.class,
|
|
|
- new TestAppContext());
|
|
|
+ new MockAppContext(0, 1, 1, 1));
|
|
|
}
|
|
|
|
|
|
@Test public void testJobCounterView() {
|
|
|
LOG.info("JobCounterView");
|
|
|
- AppContext appContext = new TestAppContext();
|
|
|
+ AppContext appContext = new MockAppContext(0, 1, 1, 1);
|
|
|
Map<String, String> params = TestAMWebApp.getJobParams(appContext);
|
|
|
WebAppTests.testPage(HsCountersPage.class, AppContext.class,
|
|
|
appContext, params);
|
|
@@ -211,7 +131,7 @@ public class TestHSWebApp {
|
|
|
|
|
|
@Test public void testJobCounterViewForKilledJob() {
|
|
|
LOG.info("JobCounterViewForKilledJob");
|
|
|
- AppContext appContext = new TestAppContext(0, 1, 1, 1, true);
|
|
|
+ AppContext appContext = new MockAppContext(0, 1, 1, 1, true);
|
|
|
Map<String, String> params = TestAMWebApp.getJobParams(appContext);
|
|
|
WebAppTests.testPage(HsCountersPage.class, AppContext.class,
|
|
|
appContext, params);
|
|
@@ -220,7 +140,7 @@ public class TestHSWebApp {
|
|
|
@Test public void testSingleCounterView() {
|
|
|
LOG.info("HsSingleCounterPage");
|
|
|
WebAppTests.testPage(HsSingleCounterPage.class, AppContext.class,
|
|
|
- new TestAppContext());
|
|
|
+ new MockAppContext(0, 1, 1, 1));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -228,7 +148,7 @@ public class TestHSWebApp {
|
|
|
LOG.info("HsLogsPage");
|
|
|
Injector injector =
|
|
|
WebAppTests.testPage(AggregatedLogsPage.class, AppContext.class,
|
|
|
- new TestAppContext());
|
|
|
+ new MockAppContext(0, 1, 1, 1));
|
|
|
PrintWriter spyPw = WebAppTests.getPrintWriter(injector);
|
|
|
verify(spyPw).write("Cannot get container logs without a ContainerId");
|
|
|
verify(spyPw).write("Cannot get container logs without a NodeId");
|
|
@@ -238,7 +158,7 @@ public class TestHSWebApp {
|
|
|
@Test
|
|
|
public void testLogsView2() throws IOException {
|
|
|
LOG.info("HsLogsPage with data");
|
|
|
- TestAppContext ctx = new TestAppContext();
|
|
|
+ MockAppContext ctx = new MockAppContext(0, 1, 1, 1);
|
|
|
Map<String, String> params = new HashMap<String, String>();
|
|
|
|
|
|
params.put(CONTAINER_ID, MRApp.newContainerId(1, 1, 333, 1)
|
|
@@ -260,7 +180,7 @@ public class TestHSWebApp {
|
|
|
@Test
|
|
|
public void testLogsViewSingle() throws IOException {
|
|
|
LOG.info("HsLogsPage with params for single log and data limits");
|
|
|
- TestAppContext ctx = new TestAppContext();
|
|
|
+ MockAppContext ctx = new MockAppContext(0, 1, 1, 1);
|
|
|
Map<String, String> params = new HashMap<String, String>();
|
|
|
|
|
|
final Configuration conf = new YarnConfiguration();
|
|
@@ -295,7 +215,7 @@ public class TestHSWebApp {
|
|
|
@Test
|
|
|
public void testLogsViewBadStartEnd() throws IOException {
|
|
|
LOG.info("HsLogsPage with bad start/end params");
|
|
|
- TestAppContext ctx = new TestAppContext();
|
|
|
+ MockAppContext ctx = new MockAppContext(0, 1, 1, 1);
|
|
|
Map<String, String> params = new HashMap<String, String>();
|
|
|
|
|
|
params.put("start", "foo");
|