Browse Source

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 năm trước cách đây
mục cha
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);
+      }
+    }
+  }
+
 }