Quellcode durchsuchen

MAPREDUCE-5084. fix coverage org.apache.hadoop.mapreduce.v2.app.webapp and org.apache.hadoop.mapreduce.v2.hs.webapp (Aleksey Gorshkov via jeagles)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1494687 13f79535-47bb-0310-9956-ffa450edef68
Jonathan Turner Eagles vor 12 Jahren
Ursprung
Commit
d8203c0ae9

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

@@ -1034,6 +1034,9 @@ Release 0.23.9 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-5084. fix coverage org.apache.hadoop.mapreduce.v2.app.webapp and
+    org.apache.hadoop.mapreduce.v2.hs.webapp (Aleksey Gorshkov via jeagles)
+
   OPTIMIZATIONS
 
     MAPREDUCE-5268. Improve history server startup performance (Karthik

+ 115 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppControllerForTest.java

@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.app.webapp;
+
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.webapp.ResponseInfo;
+import org.apache.hadoop.yarn.webapp.View;
+
+import static org.mockito.Mockito.*;
+
+/**
+ * Class AppControllerForTest overrides some methods of AppController for test
+ */
+public class AppControllerForTest extends AppController {
+  final private static Map<String, String> properties = new HashMap<String, String>();
+
+  private ResponseInfo responseInfo = new ResponseInfo();
+  private View view = new ViewForTest();
+  private Class<?> clazz;
+  private HttpServletResponse response;
+
+
+  protected AppControllerForTest(App app, Configuration configuration, RequestContext ctx) {
+    super(app, configuration, ctx);
+  }
+
+  public Class<?> getClazz() {
+    return clazz;
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> T getInstance(Class<T> cls) {
+    clazz = cls;
+    if (cls.equals(ResponseInfo.class)) {
+      return (T) responseInfo;
+    }
+    return (T) view;
+  }
+
+  public ResponseInfo getResponseInfo() {
+    return responseInfo;
+  }
+
+  public String get(String key, String defaultValue) {
+    String result = properties.get(key);
+    if (result == null) {
+      result = defaultValue;
+    }
+    return result;
+  }
+
+  public void set(String key, String value) {
+    properties.put(key, value);
+  }
+
+  public HttpServletRequest request() {
+    HttpServletRequest result = mock(HttpServletRequest.class);
+    when(result.getRemoteUser()).thenReturn("user");
+
+
+    return result;
+  }
+
+  @Override
+  public HttpServletResponse response() {
+    if (response == null) {
+      response = mock(HttpServletResponse.class);
+    }
+    return response;
+  }
+
+  public Map<String, String> getProperty() {
+    return properties;
+  }
+
+  OutputStream data = new ByteArrayOutputStream();
+  PrintWriter writer = new PrintWriter(data);
+
+  public String getData() {
+    writer.flush();
+    return data.toString();
+  }
+
+  protected PrintWriter writer() {
+    if (writer == null) {
+      writer = new PrintWriter(data);
+    }
+    return writer;
+  }
+}

+ 41 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppForTest.java

@@ -0,0 +1,41 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.mapreduce.v2.app.webapp;
+
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+
+/**
+ *   Class AppForTest publishes a methods for test
+ */
+public class AppForTest extends App {
+
+  public AppForTest(AppContext ctx) {
+    super(ctx);
+  }
+  public void setJob(Job job) {
+    super.setJob(job);
+  }
+  @Override
+  public void setTask(Task task) {
+    super.setTask(task);
+  }
+
+}

+ 49 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlockForTest.java

@@ -0,0 +1,49 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.mapreduce.v2.app.webapp;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ *    Class TasksBlockForTest overrides some methods for test
+ */
+public class TasksBlockForTest extends TasksBlock{
+  private final Map<String, String> params = new HashMap<String, String>();
+
+
+  public TasksBlockForTest(App app) {
+     super(app);
+   }
+
+  public void addParameter(String name, String value) {
+    params.put(name, value);
+  }
+  @Override
+  public String $(String key, String defaultValue) {
+    String value = params.get(key);
+    return value == null ? defaultValue : value;
+  }
+  public String url(String... parts) {
+    String result = "url://";
+    for (String string : parts) {
+      result += string +":";
+   }
+    return result;
+  }
+}

+ 251 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java

@@ -17,36 +17,66 @@
  */
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
+
+import java.io.IOException;
+import java.util.Iterator;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobACL;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.webapp.Controller.RequestContext;
+import org.apache.hadoop.yarn.webapp.MimeType;
+import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.junit.Before;
 import org.junit.Test;
+import static org.junit.Assert.*;
 
 public class TestAppController {
 
-  private AppController appController;
+  private AppControllerForTest appController;
   private RequestContext ctx;
+  private Job job;
 
   @Before
-  public void setUp() {
+  public void setUp() throws IOException {
     AppContext context = mock(AppContext.class);
     when(context.getApplicationID()).thenReturn(
         ApplicationId.newInstance(0, 0));
+    when(context.getApplicationName()).thenReturn("AppName");
+    when(context.getUser()).thenReturn("User");
+    when(context.getStartTime()).thenReturn(System.currentTimeMillis());
+    job = mock(Job.class);
+    Task task = mock(Task.class);
+
+    when(job.getTask(any(TaskId.class))).thenReturn(task);
+
+    JobId jobID = MRApps.toJobID("job_01_01");
+    when(context.getJob(jobID)).thenReturn(job);
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(true);
+
     App app = new App(context);
-    Configuration conf = new Configuration();
+    Configuration configuration = new Configuration();
     ctx = mock(RequestContext.class);
-    appController = new AppController(app, conf, ctx);
+
+    appController = new AppControllerForTest(app, configuration, ctx);
+    appController.getProperty().put(AMParams.JOB_ID, "job_01_01");
+    appController.getProperty().put(AMParams.TASK_ID, "task_01_01_m01_01");
+
   }
 
+  /**
+   * test bad request should be status 400...
+   */
   @Test
   public void testBadRequest() {
     String message = "test string";
@@ -63,8 +93,217 @@ public class TestAppController {
 
   private void verifyExpectations(String message) {
     verify(ctx).setStatus(400);
-    verify(ctx).set("app.id", "application_0_0000");
-    verify(ctx).set(eq("rm.web"), anyString());
-    verify(ctx).set("title", "Bad request: " + message);
+    assertEquals("application_0_0000", appController.getProperty()
+        .get("app.id"));
+    assertNotNull(appController.getProperty().get("rm.web"));
+    assertEquals("Bad request: " + message,
+        appController.getProperty().get("title"));
+  }
+
+  /**
+   * Test the method 'info'.
+   */
+  @Test
+  public void testInfo() {
+
+    appController.info();
+    Iterator<ResponseInfo.Item> iterator = appController.getResponseInfo()
+        .iterator();
+    ResponseInfo.Item item = iterator.next();
+    assertEquals("Application ID:", item.key);
+    assertEquals("application_0_0000", item.value);
+    item = iterator.next();
+    assertEquals("Application Name:", item.key);
+    assertEquals("AppName", item.value);
+    item = iterator.next();
+    assertEquals("User:", item.key);
+    assertEquals("User", item.value);
+
+    item = iterator.next();
+    assertEquals("Started on:", item.key);
+    item = iterator.next();
+    assertEquals("Elasped: ", item.key);
+
+  }
+
+  /**
+   *  Test method 'job'. Should print message about error or set JobPage class for rendering
+   */
+  @Test
+  public void testGetJob() {
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(false);
+
+    appController.job();
+    verify(appController.response()).setContentType(MimeType.TEXT);
+    assertEquals(
+        "Access denied: User user does not have permission to view job job_01_01",
+        appController.getData());
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(true);
+
+    appController.getProperty().remove(AMParams.JOB_ID);
+    appController.job();
+    assertEquals(
+        "Access denied: User user does not have permission to view job job_01_01Bad Request: Missing job ID",
+        appController.getData());
+
+    appController.getProperty().put(AMParams.JOB_ID, "job_01_01");
+    appController.job();
+    assertEquals(JobPage.class, appController.getClazz());
+  }
+
+  /**
+   *  Test method 'jobCounters'. Should print message about error or set CountersPage class for rendering
+   */
+  @Test
+  public void testGetJobCounters() {
+
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(false);
+
+    appController.jobCounters();
+    verify(appController.response()).setContentType(MimeType.TEXT);
+    assertEquals(
+        "Access denied: User user does not have permission to view job job_01_01",
+        appController.getData());
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(true);
+
+    appController.getProperty().remove(AMParams.JOB_ID);
+    appController.jobCounters();
+    assertEquals(
+        "Access denied: User user does not have permission to view job job_01_01Bad Request: Missing job ID",
+        appController.getData());
+
+    appController.getProperty().put(AMParams.JOB_ID, "job_01_01");
+    appController.jobCounters();
+    assertEquals(CountersPage.class, appController.getClazz());
+  }
+
+  /**
+   *  Test method 'taskCounters'. Should print message about error or set CountersPage class for rendering
+   */
+  @Test
+  public void testGetTaskCounters() {
+
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(false);
+
+    appController.taskCounters();
+    verify(appController.response()).setContentType(MimeType.TEXT);
+    assertEquals(
+        "Access denied: User user does not have permission to view job job_01_01",
+        appController.getData());
+
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(true);
+
+    appController.getProperty().remove(AMParams.TASK_ID);
+    appController.taskCounters();
+    assertEquals(
+        "Access denied: User user does not have permission to view job job_01_01missing task ID",
+        appController.getData());
+
+    appController.getProperty().put(AMParams.TASK_ID, "task_01_01_m01_01");
+    appController.taskCounters();
+    assertEquals(CountersPage.class, appController.getClazz());
+  }
+  /**
+   *  Test method 'singleJobCounter'. Should set SingleCounterPage class for rendering
+   */
+
+  @Test
+  public void testGetSingleJobCounter() throws IOException {
+    appController.singleJobCounter();
+    assertEquals(SingleCounterPage.class, appController.getClazz());
+  }
+
+  /**
+   *  Test method 'singleTaskCounter'. Should set SingleCounterPage class for rendering
+   */
+  @Test
+  public void testGetSingleTaskCounter() throws IOException {
+    appController.singleTaskCounter();
+    assertEquals(SingleCounterPage.class, appController.getClazz());
+    assertNotNull(appController.getProperty().get(AppController.COUNTER_GROUP));
+    assertNotNull(appController.getProperty().get(AppController.COUNTER_NAME));
   }
+  /**
+   *  Test method 'tasks'. Should set TasksPage class for rendering
+   */
+
+  @Test
+  public void testTasks() {
+ 
+    appController.tasks();
+ 
+    assertEquals(TasksPage.class, appController.getClazz());
+  }
+  /**
+   *  Test method 'task'. Should set TaskPage class for rendering and information for title
+   */
+  @Test
+  public void testTask() {
+ 
+    appController.task();
+    assertEquals("Attempts for task_01_01_m01_01" ,
+        appController.getProperty().get("title"));
+
+    assertEquals(TaskPage.class, appController.getClazz());
+  }
+
+  /**
+   *   Test method 'conf'. Should set JobConfPage class for rendering
+   */
+  @Test
+  public void testConfiguration() {
+ 
+    appController.conf();
+
+    assertEquals(JobConfPage.class, appController.getClazz());
+  }
+
+  /**
+   *   Test method 'conf'. Should set AttemptsPage class for rendering or print information about error
+   */
+  @Test
+  public void testAttempts() {
+
+    appController.getProperty().remove(AMParams.TASK_TYPE);
+
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(false);
+
+    appController.attempts();
+    verify(appController.response()).setContentType(MimeType.TEXT);
+    assertEquals(
+        "Access denied: User user does not have permission to view job job_01_01",
+        appController.getData());
+
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+        .thenReturn(true);
+
+    appController.getProperty().remove(AMParams.TASK_ID);
+    appController.attempts();
+    assertEquals(
+        "Access denied: User user does not have permission to view job job_01_01",
+        appController.getData());
+
+    appController.getProperty().put(AMParams.TASK_ID, "task_01_01_m01_01");
+    appController.attempts();
+    assertEquals("Bad request: missing task-type.", appController.getProperty()
+        .get("title"));
+    appController.getProperty().put(AMParams.TASK_TYPE, "m");
+
+    appController.attempts();
+    assertEquals("Bad request: missing attempt-state.", appController
+        .getProperty().get("title"));
+    appController.getProperty().put(AMParams.ATTEMPT_STATE, "State");
+
+    appController.attempts();
+
+    assertEquals(AttemptsPage.class, appController.getClazz());
+  }
+
 }

+ 168 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java

@@ -0,0 +1,168 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.app.webapp;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.JobIdPBImpl;
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.TaskIdPBImpl;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.webapp.view.BlockForTest;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
+
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
+
+public class TestBlocks {
+  private ByteArrayOutputStream data = new ByteArrayOutputStream();
+
+  /**
+   * Test rendering for ConfBlock
+   */
+  @Test
+  public void testConfigurationBlock() throws Exception {
+    AppContext ctx = mock(AppContext.class);
+    Job job = mock(Job.class);
+    Path path = new Path("conf");
+    Configuration configuration = new Configuration();
+    configuration.set("Key for test", "Value for test");
+    when(job.getConfFile()).thenReturn(path);
+    when(job.loadConfFile()).thenReturn(configuration);
+
+    when(ctx.getJob(any(JobId.class))).thenReturn(job);
+
+
+    ConfBlockForTest configurationBlock = new ConfBlockForTest(ctx);
+    PrintWriter pWriter = new PrintWriter(data);
+    Block html = new BlockForTest(new HtmlBlockForTest(), pWriter, 0, false);
+
+    configurationBlock.render(html);
+    pWriter.flush();
+    assertTrue(data.toString().contains(
+            "Sorry, can't do anything without a JobID"));
+
+    configurationBlock.addParameter(AMParams.JOB_ID, "job_01_01");
+    data.reset();
+    configurationBlock.render(html);
+    pWriter.flush();
+    assertTrue(data.toString().contains("Key for test"));
+
+    assertTrue(data.toString().contains("Value for test"));
+
+  }
+
+  /**
+   * Test rendering for TasksBlock
+   */
+  @Test
+  public void testTasksBlock() throws Exception {
+
+    ApplicationId appId = ApplicationIdPBImpl.newInstance(0, 1);
+    JobId jobId = new JobIdPBImpl();
+    jobId.setId(0);
+    jobId.setAppId(appId);
+
+    TaskId taskId = new TaskIdPBImpl();
+    taskId.setId(0);
+    taskId.setTaskType(TaskType.MAP);
+    taskId.setJobId(jobId);
+    Task task = mock(Task.class);
+    when(task.getID()).thenReturn(taskId);
+    TaskReport report = mock(TaskReport.class);
+    when(report.getProgress()).thenReturn(0.7f);
+    when(report.getTaskState()).thenReturn(TaskState.SUCCEEDED);
+    when(report.getStartTime()).thenReturn(100001L);
+    when(report.getFinishTime()).thenReturn(100011L);
+
+
+    when(task.getReport()).thenReturn(report);
+    when(task.getType()).thenReturn(TaskType.MAP);
+
+
+    Map<TaskId, Task> tasks = new HashMap<TaskId, Task>();
+    tasks.put(taskId, task);
+    AppContext ctx = mock(AppContext.class);
+    Job job = mock(Job.class);
+    when(job.getTasks()).thenReturn(tasks);
+
+
+    App app = new App(ctx);
+    app.setJob(job);
+    TasksBlockForTest taskBlock = new TasksBlockForTest(app);
+    taskBlock.addParameter(AMParams.TASK_TYPE, "m");
+
+    PrintWriter pWriter = new PrintWriter(data);
+    Block html = new BlockForTest(new HtmlBlockForTest(), pWriter, 0, false);
+
+    taskBlock.render(html);
+    pWriter.flush();
+    assertTrue(data.toString().contains("task_0_0001_m_000000"));
+    assertTrue(data.toString().contains("70.00"));
+    assertTrue(data.toString().contains("SUCCEEDED"));
+    assertTrue(data.toString().contains("100001"));
+    assertTrue(data.toString().contains("100011"));
+
+
+  }
+
+  private class ConfBlockForTest extends ConfBlock {
+    private final Map<String, String> params = new HashMap<String, String>();
+
+    public void addParameter(String name, String value) {
+      params.put(name, value);
+    }
+
+    @Override
+    public String $(String key, String defaultValue) {
+      String value = params.get(key);
+      return value == null ? defaultValue : value;
+    }
+
+    ConfBlockForTest(AppContext appCtx) {
+      super(appCtx);
+    }
+
+  }
+
+  private class HtmlBlockForTest extends HtmlBlock {
+
+    @Override
+    protected void render(Block html) {
+
+    }
+  }
+
+}

+ 32 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/ViewForTest.java

@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapreduce.v2.app.webapp;
+
+import org.apache.hadoop.yarn.webapp.View;
+
+/**
+ *  override method render() for test
+ */
+public class ViewForTest extends View{
+
+  @Override
+  public void render() {
+    
+  }
+
+}

+ 35 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/yarn/webapp/view/BlockForTest.java

@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.webapp.view;
+
+import java.io.PrintWriter;
+
+/**
+ *   BlockForTest publishes constructor for test
+ */
+public class BlockForTest extends HtmlBlock.Block {
+
+  public BlockForTest(HtmlBlock htmlBlock, PrintWriter out, int level,
+      boolean wasInline) {
+    htmlBlock.super(out, level, wasInline);
+  }
+
+  
+  
+}

+ 438 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java

@@ -0,0 +1,438 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.hs.webapp;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobACL;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
+import org.apache.hadoop.mapreduce.v2.api.records.JobState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.JobIdPBImpl;
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.TaskAttemptIdPBImpl;
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.TaskIdPBImpl;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.webapp.AMParams;
+import org.apache.hadoop.mapreduce.v2.app.webapp.App;
+import org.apache.hadoop.mapreduce.v2.app.webapp.AppForTest;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.HsTaskPage.AttemptsBlock;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.webapp.Controller.RequestContext;
+import org.apache.hadoop.yarn.webapp.Params;
+import org.apache.hadoop.yarn.webapp.View;
+import org.apache.hadoop.yarn.webapp.log.AggregatedLogsPage;
+import org.apache.hadoop.yarn.webapp.view.BlockForTest;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.*;
+
+/**
+ * Test some HtmlBlock classes
+ */
+
+public class TestBlocks {
+  private ByteArrayOutputStream data = new ByteArrayOutputStream();
+
+  /**
+   * test HsTasksBlock's rendering.
+   */
+  @Test
+  public void testHsTasksBlock() {
+
+    Task task = getTask(0);
+
+    Map<TaskId, Task> tasks = new HashMap<TaskId, Task>();
+    tasks.put(task.getID(), task);
+
+    AppContext ctx = mock(AppContext.class);
+    AppForTest app = new AppForTest(ctx);
+    Job job = mock(Job.class);
+    when(job.getTasks()).thenReturn(tasks);
+
+    app.setJob(job);
+
+    HsTasksBlockForTest block = new HsTasksBlockForTest(app);
+
+    block.addParameter(AMParams.TASK_TYPE, "r");
+
+    PrintWriter pWriter = new PrintWriter(data);
+    Block html = new BlockForTest(new HtmlBlockForTest(), pWriter, 0, false);
+
+    block.render(html);
+    pWriter.flush();
+    // should be printed information about task
+    assertTrue(data.toString().contains("task_0_0001_r_000000"));
+    assertTrue(data.toString().contains("SUCCEEDED"));
+    assertTrue(data.toString().contains("100001"));
+    assertTrue(data.toString().contains("100011"));
+    assertTrue(data.toString().contains(""));
+  }
+
+  /**
+   * test AttemptsBlock's rendering.
+   */
+  @Test
+  public void testAttemptsBlock() {
+    AppContext ctx = mock(AppContext.class);
+    AppForTest app = new AppForTest(ctx);
+
+    Task task = getTask(0);
+    Map<TaskAttemptId, TaskAttempt> attempts = new HashMap<TaskAttemptId, TaskAttempt>();
+    TaskAttempt attempt = mock(TaskAttempt.class);
+    TaskAttemptId taId = new TaskAttemptIdPBImpl();
+    taId.setId(0);
+    taId.setTaskId(task.getID());
+    when(attempt.getID()).thenReturn(taId);
+    when(attempt.getNodeHttpAddress()).thenReturn("Node address");
+
+    ApplicationId appId = ApplicationIdPBImpl.newInstance(0, 5);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptIdPBImpl.newInstance(appId, 1);
+
+    ContainerId containerId = ContainerIdPBImpl.newInstance(appAttemptId, 1);
+    when(attempt.getAssignedContainerID()).thenReturn(containerId);
+
+    when(attempt.getAssignedContainerMgrAddress()).thenReturn(
+            "assignedContainerMgrAddress");
+    when(attempt.getNodeRackName()).thenReturn("nodeRackName");
+    when(attempt.getLaunchTime()).thenReturn(100002L);
+    when(attempt.getFinishTime()).thenReturn(100012L);
+    when(attempt.getShuffleFinishTime()).thenReturn(100010L);
+    when(attempt.getSortFinishTime()).thenReturn(100011L);
+    when(attempt.getState()).thenReturn(TaskAttemptState.SUCCEEDED);
+
+    attempts.put(taId, attempt);
+    when(task.getAttempts()).thenReturn(attempts);
+
+    app.setTask(task);
+    Job job = mock(Job.class);
+    when(job.getUserName()).thenReturn("User");
+    app.setJob(job);
+
+    AttemptsBlockForTest block = new AttemptsBlockForTest(app);
+    block.addParameter(AMParams.TASK_TYPE, "r");
+
+    PrintWriter pWriter = new PrintWriter(data);
+    Block html = new BlockForTest(new HtmlBlockForTest(), pWriter, 0, false);
+
+    block.render(html);
+    pWriter.flush();
+    // should be printed information about attempts
+    assertTrue(data.toString().contains("0 attempt_0_0001_r_000000_0"));
+    assertTrue(data.toString().contains("SUCCEEDED"));
+    assertTrue(data.toString().contains(
+            "_0005_01_000001:attempt_0_0001_r_000000_0:User:"));
+    assertTrue(data.toString().contains("100002"));
+    assertTrue(data.toString().contains("100010"));
+    assertTrue(data.toString().contains("100011"));
+    assertTrue(data.toString().contains("100012"));
+  }
+
+  /**
+   * test HsJobsBlock's rendering.
+   */
+  @Test
+  public void testHsJobsBlock() {
+    AppContext ctx = mock(AppContext.class);
+    Map<JobId, Job> jobs = new HashMap<JobId, Job>();
+    Job job = getJob();
+    jobs.put(job.getID(), job);
+    when(ctx.getAllJobs()).thenReturn(jobs);
+
+    HsJobsBlock block = new HsJobsBlockForTest(ctx);
+    PrintWriter pWriter = new PrintWriter(data);
+    Block html = new BlockForTest(new HtmlBlockForTest(), pWriter, 0, false);
+    block.render(html);
+
+    pWriter.flush();
+    assertTrue(data.toString().contains("JobName"));
+    assertTrue(data.toString().contains("UserName"));
+    assertTrue(data.toString().contains("QueueName"));
+    assertTrue(data.toString().contains("SUCCEEDED"));
+  }
+  /**
+   * test HsController
+   */
+
+  @Test
+  public void testHsController() throws Exception {
+    AppContext ctx = mock(AppContext.class);
+    ApplicationId appId = ApplicationIdPBImpl.newInstance(0,5);
+    
+    when(ctx.getApplicationID()).thenReturn(appId);
+
+    AppForTest app = new AppForTest(ctx);
+    Configuration config = new Configuration();
+    RequestContext requestCtx = mock(RequestContext.class);
+    HsControllerForTest controller = new HsControllerForTest(app, config,
+            requestCtx);
+    controller.index();
+    assertEquals("JobHistory", controller.get(Params.TITLE, ""));
+    assertEquals(HsJobPage.class, controller.jobPage());
+    assertEquals(HsCountersPage.class, controller.countersPage());
+    assertEquals(HsTasksPage.class, controller.tasksPage());
+    assertEquals(HsTaskPage.class, controller.taskPage());
+    assertEquals(HsAttemptsPage.class, controller.attemptsPage());
+
+    controller.set(AMParams.JOB_ID, "job_01_01");
+    controller.set(AMParams.TASK_ID, "task_01_01_m01_01");
+    controller.set(AMParams.TASK_TYPE, "m");
+    controller.set(AMParams.ATTEMPT_STATE, "State");
+
+    Job job = mock(Job.class);
+    Task task = mock(Task.class);
+    when(job.getTask(any(TaskId.class))).thenReturn(task);
+    JobId jobID = MRApps.toJobID("job_01_01");
+    when(ctx.getJob(jobID)).thenReturn(job);
+    when(job.checkAccess(any(UserGroupInformation.class), any(JobACL.class)))
+            .thenReturn(true);
+
+    controller.job();
+    assertEquals(HsJobPage.class, controller.getClazz());
+    controller.jobCounters();
+    assertEquals(HsCountersPage.class, controller.getClazz());
+    controller.taskCounters();
+    assertEquals(HsCountersPage.class, controller.getClazz());
+    controller.tasks();
+    assertEquals(HsTasksPage.class, controller.getClazz());
+    controller.task();
+    assertEquals(HsTaskPage.class, controller.getClazz());
+    controller.attempts();
+    assertEquals(HsAttemptsPage.class, controller.getClazz());
+
+    assertEquals(HsConfPage.class, controller.confPage());
+    assertEquals(HsAboutPage.class, controller.aboutPage());
+    controller.about();
+    assertEquals(HsAboutPage.class, controller.getClazz());
+    controller.logs();
+    assertEquals(HsLogsPage.class, controller.getClazz());
+    controller.nmlogs();
+    assertEquals(AggregatedLogsPage.class, controller.getClazz());
+
+    assertEquals(HsSingleCounterPage.class, controller.singleCounterPage());
+    controller.singleJobCounter();
+    assertEquals(HsSingleCounterPage.class, controller.getClazz());
+    controller.singleTaskCounter();
+    assertEquals(HsSingleCounterPage.class, controller.getClazz());
+
+
+  }
+
+  private static class HsControllerForTest extends HsController {
+
+    static private Map<String, String> params = new HashMap<String, String>();
+    private Class<?> clazz;
+    ByteArrayOutputStream data = new ByteArrayOutputStream();
+
+    public void set(String name, String value) {
+      params.put(name, value);
+    }
+
+    public String get(String key, String defaultValue) {
+      String value = params.get(key);
+      return value == null ? defaultValue : value;
+    }
+
+    HsControllerForTest(App app, Configuration configuration, RequestContext ctx) {
+      super(app, configuration, ctx);
+    }
+
+    @Override
+    public HttpServletRequest request() {
+      HttpServletRequest result = mock(HttpServletRequest.class);
+      when(result.getRemoteUser()).thenReturn("User");
+      return result;
+    }
+
+    public HttpServletResponse response() {
+      HttpServletResponse result = mock(HttpServletResponse.class);
+      try {
+        when(result.getWriter()).thenReturn(new PrintWriter(data));
+      } catch (IOException ignored) {
+
+      }
+
+      return result;
+    }
+
+    protected void render(Class<? extends View> cls) {
+      clazz = cls;
+    }
+
+    public Class<?> getClazz() {
+      return clazz;
+    }
+  }
+
+  private Job getJob() {
+    Job job = mock(Job.class);
+
+    JobId jobId = new JobIdPBImpl();
+
+    ApplicationId appId = ApplicationIdPBImpl.newInstance(System.currentTimeMillis(),4);
+    jobId.setAppId(appId);
+    jobId.setId(1);
+    when(job.getID()).thenReturn(jobId);
+
+    JobReport report = mock(JobReport.class);
+    when(report.getStartTime()).thenReturn(100010L);
+    when(report.getFinishTime()).thenReturn(100015L);
+
+    when(job.getReport()).thenReturn(report);
+    when(job.getName()).thenReturn("JobName");
+    when(job.getUserName()).thenReturn("UserName");
+    when(job.getQueueName()).thenReturn("QueueName");
+    when(job.getState()).thenReturn(JobState.SUCCEEDED);
+    when(job.getTotalMaps()).thenReturn(3);
+    when(job.getCompletedMaps()).thenReturn(2);
+    when(job.getTotalReduces()).thenReturn(2);
+    when(job.getCompletedReduces()).thenReturn(1);
+    when(job.getCompletedReduces()).thenReturn(1);
+    return job;
+  }
+
+
+  private Task getTask(long timestamp) {
+    
+    JobId jobId = new JobIdPBImpl();
+    jobId.setId(0);
+    jobId.setAppId(ApplicationIdPBImpl.newInstance(timestamp,1));
+
+    TaskId taskId = new TaskIdPBImpl();
+    taskId.setId(0);
+    taskId.setTaskType(TaskType.REDUCE);
+    taskId.setJobId(jobId);
+    Task task = mock(Task.class);
+    when(task.getID()).thenReturn(taskId);
+    TaskReport report = mock(TaskReport.class);
+    when(report.getProgress()).thenReturn(0.7f);
+    when(report.getTaskState()).thenReturn(TaskState.SUCCEEDED);
+    when(report.getStartTime()).thenReturn(100001L);
+    when(report.getFinishTime()).thenReturn(100011L);
+
+    when(task.getReport()).thenReturn(report);
+    when(task.getType()).thenReturn(TaskType.REDUCE);
+    return task;
+  }
+
+  private class HsJobsBlockForTest extends HsJobsBlock {
+    HsJobsBlockForTest(AppContext appCtx) {
+      super(appCtx);
+    }
+
+    @Override
+    public String url(String... parts) {
+      String result = "url://";
+      for (String string : parts) {
+        result += string + ":";
+      }
+      return result;
+    }
+
+  }
+
+  private class AttemptsBlockForTest extends AttemptsBlock {
+    private final Map<String, String> params = new HashMap<String, String>();
+
+    public void addParameter(String name, String value) {
+      params.put(name, value);
+    }
+
+    public String $(String key, String defaultValue) {
+      String value = params.get(key);
+      return value == null ? defaultValue : value;
+    }
+
+    public AttemptsBlockForTest(App ctx) {
+      super(ctx);
+    }
+
+    @Override
+    public String url(String... parts) {
+      String result = "url://";
+      for (String string : parts) {
+        result += string + ":";
+      }
+      return result;
+    }
+
+  }
+
+  private class HsTasksBlockForTest extends HsTasksBlock {
+    private final Map<String, String> params = new HashMap<String, String>();
+
+    public void addParameter(String name, String value) {
+      params.put(name, value);
+    }
+
+    public String $(String key, String defaultValue) {
+      String value = params.get(key);
+      return value == null ? defaultValue : value;
+    }
+
+    @Override
+    public String url(String... parts) {
+      String result = "url://";
+      for (String string : parts) {
+        result += string + ":";
+      }
+      return result;
+    }
+
+    public HsTasksBlockForTest(App app) {
+      super(app);
+    }
+  }
+
+  private class HtmlBlockForTest extends HtmlBlock {
+
+    @Override
+    protected void render(Block html) {
+
+    }
+  }
+
+}