Bladeren bron

YARN-2591. Fixed AHSWebServices to return FORBIDDEN(403) if the request user doesn't have access to the history data. Contributed by Zhijie Shen

Jian He 10 jaren geleden
bovenliggende
commit
c05b581a55

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AuthorizationException.java

@@ -29,7 +29,7 @@ import org.apache.hadoop.security.AccessControlException;
  * 
  * This class <em>does not</em> provide the stack trace for security purposes.
  */
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN"})
 @InterfaceStability.Evolving
 public class AuthorizationException extends AccessControlException {
   private static final long serialVersionUID = 1L;

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

@@ -766,6 +766,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2726. CapacityScheduler should explicitly log when an accessible
     label has no capacity. (Wangda Tan via xgong)
 
+    YARN-2591. Fixed AHSWebServices to return FORBIDDEN(403) if the request user
+    doesn't have access to the history data. (Zhijie Shen via jianhe)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java

@@ -26,6 +26,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -504,8 +505,8 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
           app.appReport.setOriginalTrackingUrl(appAttempt.getOriginalTrackingUrl());
         }
       }
-    } catch (YarnException e) {
-      // YarnExcetpion is thrown because the user doesn't have access
+    } catch (AuthorizationException e) {
+      // AuthorizationException is thrown because the user doesn't have access
       app.appReport.setDiagnostics(null);
       app.appReport.setCurrentApplicationAttemptId(null);
     }
@@ -554,7 +555,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
          if (!aclsManager.checkAccess(UserGroupInformation.getCurrentUser(),
              ApplicationAccessType.VIEW_APP, app.appReport.getUser(),
              app.appReport.getApplicationId())) {
-           throw new YarnException("User "
+           throw new AuthorizationException("User "
                + UserGroupInformation.getCurrentUser().getShortUserName()
                + " does not have privilage to see this application "
                + app.appReport.getApplicationId());

+ 27 - 37
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.applicationhistoryservice;
 
-import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Collection;
@@ -28,6 +27,7 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -74,8 +74,13 @@ public class TestApplicationHistoryManagerOnTimelineStore {
 
   @BeforeClass
   public static void prepareStore() throws Exception {
-    store = new MemoryTimelineStore();
-    prepareTimelineStore(store);
+    store = createStore(SCALE);
+  }
+
+  public static TimelineStore createStore(int scale) throws Exception {
+    TimelineStore store = new MemoryTimelineStore();
+    prepareTimelineStore(store, scale);
+    return store;
   }
 
   @Before
@@ -117,9 +122,9 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     }
   }
 
-  private static void prepareTimelineStore(TimelineStore store)
+  private static void prepareTimelineStore(TimelineStore store, int scale)
       throws Exception {
-    for (int i = 1; i <= SCALE; ++i) {
+    for (int i = 1; i <= scale; ++i) {
       TimelineEntities entities = new TimelineEntities();
       ApplicationId appId = ApplicationId.newInstance(0, i);
       if (i == 2) {
@@ -128,13 +133,13 @@ public class TestApplicationHistoryManagerOnTimelineStore {
         entities.addEntity(createApplicationTimelineEntity(appId, false));
       }
       store.put(entities);
-      for (int j = 1; j <= SCALE; ++j) {
+      for (int j = 1; j <= scale; ++j) {
         entities = new TimelineEntities();
         ApplicationAttemptId appAttemptId =
             ApplicationAttemptId.newInstance(appId, j);
         entities.addEntity(createAppAttemptTimelineEntity(appAttemptId));
         store.put(entities);
-        for (int k = 1; k <= SCALE; ++k) {
+        for (int k = 1; k <= scale; ++k) {
           entities = new TimelineEntities();
           ContainerId containerId = ContainerId.newInstance(appAttemptId, k);
           entities.addEntity(createContainerEntity(containerId));
@@ -223,13 +228,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
           // The exception is expected
           Assert.fail();
         }
-      } catch (UndeclaredThrowableException e) {
+      } catch (AuthorizationException e) {
         if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
-          if (e.getCause().getMessage().contains(
-              "does not have privilage to see this application")) {
-            // The exception is expected
-            return;
-          }
+          // The exception is expected
+          return;
         }
         throw e;
       }
@@ -269,13 +271,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
           // The exception is expected
           Assert.fail();
         }
-      } catch (UndeclaredThrowableException e) {
+      } catch (AuthorizationException e) {
         if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
-          if (e.getCause().getMessage().contains(
-              "does not have privilage to see this application")) {
-            // The exception is expected
-            return;
-          }
+          // The exception is expected
+          return;
         }
         throw e;
       }
@@ -324,13 +323,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
           // The exception is expected
           Assert.fail();
         }
-      } catch (UndeclaredThrowableException e) {
+      } catch (AuthorizationException e) {
         if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
-          if (e.getCause().getMessage().contains(
-              "does not have privilage to see this application")) {
-            // The exception is expected
-            return;
-          }
+          // The exception is expected
+          return;
         }
         throw e;
       }
@@ -359,13 +355,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
           // The exception is expected
           Assert.fail();
         }
-      } catch (UndeclaredThrowableException e) {
+      } catch (AuthorizationException e) {
         if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
-          if (e.getCause().getMessage().contains(
-              "does not have privilage to see this application")) {
-            // The exception is expected
-            return;
-          }
+          // The exception is expected
+          return;
         }
         throw e;
       }
@@ -394,13 +387,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
           // The exception is expected
           Assert.fail();
         }
-      } catch (UndeclaredThrowableException e) {
+      } catch (AuthorizationException e) {
         if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
-          if (e.getCause().getMessage().contains(
-              "does not have privilage to see this application")) {
-            // The exception is expected
-            return;
-          }
+          // The exception is expected
+          return;
         }
         throw e;
       }

+ 122 - 50
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java

@@ -21,36 +21,51 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import javax.ws.rs.core.MediaType;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Properties;
 
-import org.junit.Assert;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.lib.StaticUserWebFilter.StaticUserFilter;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 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.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.api.ApplicationContext;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.MemoryApplicationHistoryStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManagerOnTimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.TestApplicationHistoryManagerOnTimelineStore;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
+import org.apache.hadoop.yarn.server.timeline.TimelineStore;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
-import org.junit.Before;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;
+import com.google.inject.Singleton;
 import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
@@ -61,9 +76,40 @@ import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.test.framework.JerseyTest;
 import com.sun.jersey.test.framework.WebAppDescriptor;
 
+@RunWith(Parameterized.class)
 public class TestAHSWebServices extends JerseyTest {
 
-  private static ApplicationHistoryManager ahManager;
+  private static ApplicationHistoryManagerOnTimelineStore historyManager;
+  private static final String[] USERS = new String[] { "foo" , "bar" };
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    TimelineStore store =
+        TestApplicationHistoryManagerOnTimelineStore.createStore(5);
+    TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
+    TimelineDataManager dataManager =
+        new TimelineDataManager(store, aclsManager);
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    conf.set(YarnConfiguration.YARN_ADMIN_ACL, "foo");
+    ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
+    historyManager =
+        new ApplicationHistoryManagerOnTimelineStore(dataManager, appAclsManager);
+    historyManager.init(conf);
+    historyManager.start();
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    if (historyManager != null) {
+      historyManager.stop();
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> rounds() {
+    return Arrays.asList(new Object[][] { { 0 }, { 1 } });
+  }
 
   private Injector injector = Guice.createInjector(new ServletModule() {
 
@@ -72,16 +118,25 @@ public class TestAHSWebServices extends JerseyTest {
       bind(JAXBContextResolver.class);
       bind(AHSWebServices.class);
       bind(GenericExceptionHandler.class);
-      try {
-        ahManager = mockApplicationHistoryManager();
-      } catch (Exception e) {
-        Assert.fail();
-      }
-      bind(ApplicationContext.class).toInstance(ahManager);
+      bind(ApplicationContext.class).toInstance(historyManager);
       serve("/*").with(GuiceContainer.class);
+      filter("/*").through(TestSimpleAuthFilter.class);
     }
   });
 
+  @Singleton
+  public static class TestSimpleAuthFilter extends AuthenticationFilter {
+    @Override
+    protected Properties getConfiguration(String configPrefix,
+        FilterConfig filterConfig) throws ServletException {
+      Properties properties =
+          super.getConfiguration(configPrefix, filterConfig);
+      properties.put(AuthenticationFilter.AUTH_TYPE, "simple");
+      properties.put(PseudoAuthenticationHandler.ANONYMOUS_ALLOWED, "false");
+      return properties;
+    }
+  }
+
   public class GuiceServletConfig extends GuiceServletContextListener {
 
     @Override
@@ -90,28 +145,15 @@ public class TestAHSWebServices extends JerseyTest {
     }
   }
 
-  private ApplicationHistoryManager mockApplicationHistoryManager()
-      throws Exception {
-    ApplicationHistoryStore store = new MemoryApplicationHistoryStore();
-    TestAHSWebApp testAHSWebApp = new TestAHSWebApp();
-    testAHSWebApp.setApplicationHistoryStore(store);
-    ApplicationHistoryManager ahManager =
-        testAHSWebApp.mockApplicationHistoryManager(5, 5, 5);
-    return ahManager;
-  }
+  private int round;
 
-  public TestAHSWebServices() {
+  public TestAHSWebServices(int round) {
     super(new WebAppDescriptor.Builder(
       "org.apache.hadoop.yarn.server.applicationhistoryservice.webapp")
       .contextListenerClass(GuiceServletConfig.class)
       .filterClass(com.google.inject.servlet.GuiceFilter.class)
       .contextPath("jersey-guice-filter").servletPath("/").build());
-  }
-
-  @Before
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
+    this.round = round;
   }
 
   @Test
@@ -121,6 +163,7 @@ public class TestAHSWebServices extends JerseyTest {
     try {
       responseStr =
           r.path("ws").path("v1").path("applicationhistory").path("bogus")
+            .queryParam("user.name", USERS[round])
             .accept(MediaType.APPLICATION_JSON).get(String.class);
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
@@ -137,7 +180,8 @@ public class TestAHSWebServices extends JerseyTest {
     WebResource r = resource();
     String responseStr = "";
     try {
-      responseStr = r.accept(MediaType.APPLICATION_JSON).get(String.class);
+      responseStr = r.queryParam("user.name", USERS[round])
+          .accept(MediaType.APPLICATION_JSON).get(String.class);
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
@@ -154,6 +198,7 @@ public class TestAHSWebServices extends JerseyTest {
     try {
       responseStr =
           r.path("ws").path("v1").path("applicationhistory")
+            .queryParam("user.name", USERS[round])
             .accept(MediaType.TEXT_PLAIN).get(String.class);
       fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
@@ -171,6 +216,7 @@ public class TestAHSWebServices extends JerseyTest {
     ClientResponse response =
         r.path("ws").path("v1").path("applicationhistory").path("apps")
           .queryParam("state", YarnApplicationState.FINISHED.toString())
+          .queryParam("user.name", USERS[round])
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
@@ -187,18 +233,21 @@ public class TestAHSWebServices extends JerseyTest {
     WebResource r = resource();
     ClientResponse response =
         r.path("ws").path("v1").path("applicationhistory").path("apps")
-          .path(appId.toString()).accept(MediaType.APPLICATION_JSON)
+          .path(appId.toString())
+          .queryParam("user.name", USERS[round])
+          .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject app = json.getJSONObject("app");
     assertEquals(appId.toString(), app.getString("appId"));
-    assertEquals(appId.toString(), app.get("name"));
-    assertEquals(appId.toString(), app.get("diagnosticsInfo"));
+    assertEquals("test app", app.get("name"));
+    assertEquals(round == 0 ? "test diagnostics info" : "",
+        app.get("diagnosticsInfo"));
     assertEquals("test queue", app.get("queue"));
-    assertEquals("test user", app.get("user"));
-    assertEquals("test type", app.get("type"));
+    assertEquals("user1", app.get("user"));
+    assertEquals("test app type", app.get("type"));
     assertEquals(FinalApplicationStatus.UNDEFINED.toString(),
       app.get("finalAppStatus"));
     assertEquals(YarnApplicationState.FINISHED.toString(), app.get("appState"));
@@ -211,7 +260,13 @@ public class TestAHSWebServices extends JerseyTest {
     ClientResponse response =
         r.path("ws").path("v1").path("applicationhistory").path("apps")
           .path(appId.toString()).path("appattempts")
+          .queryParam("user.name", USERS[round])
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    if (round == 1) {
+      assertEquals(
+          Status.FORBIDDEN, response.getClientResponseStatus());
+      return;
+    }
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
@@ -230,15 +285,22 @@ public class TestAHSWebServices extends JerseyTest {
     ClientResponse response =
         r.path("ws").path("v1").path("applicationhistory").path("apps")
           .path(appId.toString()).path("appattempts")
-          .path(appAttemptId.toString()).accept(MediaType.APPLICATION_JSON)
+          .path(appAttemptId.toString())
+          .queryParam("user.name", USERS[round])
+          .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
+    if (round == 1) {
+      assertEquals(
+          Status.FORBIDDEN, response.getClientResponseStatus());
+      return;
+    }
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject appAttempt = json.getJSONObject("appAttempt");
     assertEquals(appAttemptId.toString(), appAttempt.getString("appAttemptId"));
-    assertEquals(appAttemptId.toString(), appAttempt.getString("host"));
-    assertEquals(appAttemptId.toString(),
+    assertEquals("test host", appAttempt.getString("host"));
+    assertEquals("test diagnostics info",
       appAttempt.getString("diagnosticsInfo"));
     assertEquals("test tracking url", appAttempt.getString("trackingUrl"));
     assertEquals(YarnApplicationAttemptState.FINISHED.toString(),
@@ -255,7 +317,13 @@ public class TestAHSWebServices extends JerseyTest {
         r.path("ws").path("v1").path("applicationhistory").path("apps")
           .path(appId.toString()).path("appattempts")
           .path(appAttemptId.toString()).path("containers")
+          .queryParam("user.name", USERS[round])
           .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    if (round == 1) {
+      assertEquals(
+          Status.FORBIDDEN, response.getClientResponseStatus());
+      return;
+    }
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
@@ -265,7 +333,6 @@ public class TestAHSWebServices extends JerseyTest {
     assertEquals("incorrect number of elements", 5, array.length());
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   public void testSingleContainer() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
@@ -277,26 +344,31 @@ public class TestAHSWebServices extends JerseyTest {
         r.path("ws").path("v1").path("applicationhistory").path("apps")
           .path(appId.toString()).path("appattempts")
           .path(appAttemptId.toString()).path("containers")
-          .path(containerId.toString()).accept(MediaType.APPLICATION_JSON)
+          .path(containerId.toString())
+          .queryParam("user.name", USERS[round])
+          .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
+    if (round == 1) {
+      assertEquals(
+          Status.FORBIDDEN, response.getClientResponseStatus());
+      return;
+    }
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     JSONObject json = response.getEntity(JSONObject.class);
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject container = json.getJSONObject("container");
     assertEquals(containerId.toString(), container.getString("containerId"));
-    assertEquals(containerId.toString(), container.getString("diagnosticsInfo"));
-    assertEquals("0", container.getString("allocatedMB"));
-    assertEquals("0", container.getString("allocatedVCores"));
-    assertEquals(NodeId.newInstance("localhost", 0).toString(),
+    assertEquals("test diagnostics info", container.getString("diagnosticsInfo"));
+    assertEquals("-1", container.getString("allocatedMB"));
+    assertEquals("-1", container.getString("allocatedVCores"));
+    assertEquals(NodeId.newInstance("test host", 100).toString(),
       container.getString("assignedNodeId"));
-    assertEquals(Priority.newInstance(containerId.getId()).toString(),
-      container.getString("priority"));
+    assertEquals("-1", container.getString("priority"));
     Configuration conf = new YarnConfiguration();
     assertEquals(WebAppUtils.getHttpSchemePrefix(conf) +
         WebAppUtils.getAHSWebAppURLWithoutScheme(conf) +
-        "/applicationhistory/logs/localhost:0/container_0_0001_01_000001/" +
-        "container_0_0001_01_000001/test user",
-        container.getString("logUrl"));
+        "/applicationhistory/logs/test host:100/container_0_0001_01_000001/" +
+        "container_0_0001_01_000001/user1", container.getString("logUrl"));
     assertEquals(ContainerState.COMPLETE.toString(),
       container.getString("containerState"));
   }

+ 25 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.webapp;
 
+import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Collection;
@@ -29,6 +30,7 @@ import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.WebApplicationException;
 
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -46,6 +48,7 @@ import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 public class WebServices {
@@ -151,7 +154,7 @@ public class WebServices {
         });
       }
     } catch (Exception e) {
-      throw new WebApplicationException(e);
+      rewrapAndThrowException(e);
     }
     for (ApplicationReport appReport : appReports) {
 
@@ -221,7 +224,7 @@ public class WebServices {
         });
       }
     } catch (Exception e) {
-      throw new WebApplicationException(e);
+      rewrapAndThrowException(e);
     }
     if (app == null) {
       throw new NotFoundException("app with id: " + appId + " not found");
@@ -247,7 +250,7 @@ public class WebServices {
         });
       }
     } catch (Exception e) {
-      throw new WebApplicationException(e);
+      rewrapAndThrowException(e);
     }
     AppAttemptsInfo appAttemptsInfo = new AppAttemptsInfo();
     for (ApplicationAttemptReport appAttemptReport : appAttemptReports) {
@@ -278,7 +281,7 @@ public class WebServices {
         });
       }
     } catch (Exception e) {
-      throw new WebApplicationException(e);
+      rewrapAndThrowException(e);
     }
     if (appAttempt == null) {
       throw new NotFoundException("app attempt with id: " + appAttemptId
@@ -307,7 +310,7 @@ public class WebServices {
         });
       }
     } catch (Exception e) {
-      throw new WebApplicationException(e);
+      rewrapAndThrowException(e);
     }
     ContainersInfo containersInfo = new ContainersInfo();
     for (ContainerReport containerReport : containerReports) {
@@ -339,7 +342,7 @@ public class WebServices {
         });
       }
     } catch (Exception e) {
-      throw new WebApplicationException(e);
+      rewrapAndThrowException(e);
     }
     if (container == null) {
       throw new NotFoundException("container with id: " + containerId
@@ -441,4 +444,20 @@ public class WebServices {
     return callerUGI;
   }
 
+  private static void rewrapAndThrowException(Exception e) {
+    if (e instanceof UndeclaredThrowableException) {
+      if (e.getCause() instanceof AuthorizationException) {
+        throw new ForbiddenException(e.getCause());
+      } else {
+        throw new WebApplicationException(e.getCause());
+      }
+    } else {
+      if (e instanceof AuthorizationException) {
+        throw new ForbiddenException(e);
+      } else {
+        throw new WebApplicationException(e);
+      }
+    }
+  }
+
 }