|
@@ -18,10 +18,16 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.applicationhistoryservice;
|
|
|
|
|
|
+import java.lang.reflect.UndeclaredThrowableException;
|
|
|
+import java.security.PrivilegedExceptionAction;
|
|
|
+import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.HashMap;
|
|
|
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.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
@@ -42,40 +48,75 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
|
|
|
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
|
|
|
import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
|
|
|
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
|
import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore;
|
|
|
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.junit.AfterClass;
|
|
|
+import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
+import org.junit.Before;
|
|
|
import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
|
+import org.junit.runner.RunWith;
|
|
|
+import org.junit.runners.Parameterized;
|
|
|
+import org.junit.runners.Parameterized.Parameters;
|
|
|
|
|
|
+@RunWith(Parameterized.class)
|
|
|
public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
|
|
|
- private static ApplicationHistoryManagerOnTimelineStore historyManager;
|
|
|
private static final int SCALE = 5;
|
|
|
+ private static TimelineStore store;
|
|
|
+
|
|
|
+ private ApplicationHistoryManagerOnTimelineStore historyManager;
|
|
|
+ private UserGroupInformation callerUGI;
|
|
|
+ private Configuration conf;
|
|
|
|
|
|
@BeforeClass
|
|
|
- public static void setup() throws Exception {
|
|
|
- YarnConfiguration conf = new YarnConfiguration();
|
|
|
- TimelineStore store = new MemoryTimelineStore();
|
|
|
+ public static void prepareStore() throws Exception {
|
|
|
+ store = new MemoryTimelineStore();
|
|
|
prepareTimelineStore(store);
|
|
|
- TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Before
|
|
|
+ public void setup() throws Exception {
|
|
|
+ // Only test the ACLs of the generic history
|
|
|
+ TimelineACLsManager aclsManager = new TimelineACLsManager(new YarnConfiguration());
|
|
|
TimelineDataManager dataManager =
|
|
|
new TimelineDataManager(store, aclsManager);
|
|
|
- historyManager = new ApplicationHistoryManagerOnTimelineStore(dataManager);
|
|
|
+ ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
|
|
|
+ historyManager =
|
|
|
+ new ApplicationHistoryManagerOnTimelineStore(dataManager, appAclsManager);
|
|
|
historyManager.init(conf);
|
|
|
historyManager.start();
|
|
|
}
|
|
|
|
|
|
- @AfterClass
|
|
|
- public static void tearDown() {
|
|
|
+ @After
|
|
|
+ public void tearDown() {
|
|
|
if (historyManager != null) {
|
|
|
historyManager.stop();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Parameters
|
|
|
+ public static Collection<Object[]> callers() {
|
|
|
+ // user1 is the owner
|
|
|
+ // user2 is the authorized user
|
|
|
+ // user3 is the unauthorized user
|
|
|
+ // admin is the admin acl
|
|
|
+ return Arrays.asList(
|
|
|
+ new Object[][] { { "" }, { "user1" }, { "user2" }, { "user3" }, { "admin" } });
|
|
|
+ }
|
|
|
+
|
|
|
+ public TestApplicationHistoryManagerOnTimelineStore(String caller) {
|
|
|
+ conf = new YarnConfiguration();
|
|
|
+ if (!caller.equals("")) {
|
|
|
+ callerUGI = UserGroupInformation.createRemoteUser(caller, AuthMethod.SIMPLE);
|
|
|
+ conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
|
|
|
+ conf.set(YarnConfiguration.YARN_ADMIN_ACL, "admin");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private static void prepareTimelineStore(TimelineStore store)
|
|
|
throws Exception {
|
|
|
for (int i = 1; i <= SCALE; ++i) {
|
|
@@ -101,23 +142,46 @@ public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
|
|
|
@Test
|
|
|
public void testGetApplicationReport() throws Exception {
|
|
|
- ApplicationId appId = ApplicationId.newInstance(0, 1);
|
|
|
- ApplicationReport app = historyManager.getApplication(appId);
|
|
|
+ final ApplicationId appId = ApplicationId.newInstance(0, 1);
|
|
|
+ ApplicationReport app;
|
|
|
+ if (callerUGI == null) {
|
|
|
+ app = historyManager.getApplication(appId);
|
|
|
+ } else {
|
|
|
+ app =
|
|
|
+ callerUGI.doAs(new PrivilegedExceptionAction<ApplicationReport> () {
|
|
|
+ @Override
|
|
|
+ public ApplicationReport run() throws Exception {
|
|
|
+ return historyManager.getApplication(appId);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
Assert.assertNotNull(app);
|
|
|
Assert.assertEquals(appId, app.getApplicationId());
|
|
|
Assert.assertEquals("test app", app.getName());
|
|
|
Assert.assertEquals("test app type", app.getApplicationType());
|
|
|
- Assert.assertEquals("test user", app.getUser());
|
|
|
+ Assert.assertEquals("user1", app.getUser());
|
|
|
Assert.assertEquals("test queue", app.getQueue());
|
|
|
Assert.assertEquals(Integer.MAX_VALUE + 2L, app.getStartTime());
|
|
|
Assert.assertEquals(Integer.MAX_VALUE + 3L, app.getFinishTime());
|
|
|
Assert.assertTrue(Math.abs(app.getProgress() - 1.0F) < 0.0001);
|
|
|
- Assert.assertEquals("test host", app.getHost());
|
|
|
- Assert.assertEquals(-100, app.getRpcPort());
|
|
|
- Assert.assertEquals("test tracking url", app.getTrackingUrl());
|
|
|
- Assert.assertEquals("test original tracking url",
|
|
|
- app.getOriginalTrackingUrl());
|
|
|
- Assert.assertEquals("test diagnostics info", app.getDiagnostics());
|
|
|
+ if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
|
|
|
+ Assert.assertEquals(ApplicationAttemptId.newInstance(appId, -1),
|
|
|
+ app.getCurrentApplicationAttemptId());
|
|
|
+ Assert.assertEquals(null, app.getHost());
|
|
|
+ Assert.assertEquals(-1, app.getRpcPort());
|
|
|
+ Assert.assertEquals(null, app.getTrackingUrl());
|
|
|
+ Assert.assertEquals(null, app.getOriginalTrackingUrl());
|
|
|
+ Assert.assertEquals(null, app.getDiagnostics());
|
|
|
+ } else {
|
|
|
+ Assert.assertEquals(ApplicationAttemptId.newInstance(appId, 1),
|
|
|
+ app.getCurrentApplicationAttemptId());
|
|
|
+ Assert.assertEquals("test host", app.getHost());
|
|
|
+ Assert.assertEquals(-100, app.getRpcPort());
|
|
|
+ Assert.assertEquals("test tracking url", app.getTrackingUrl());
|
|
|
+ Assert.assertEquals("test original tracking url",
|
|
|
+ app.getOriginalTrackingUrl());
|
|
|
+ Assert.assertEquals("test diagnostics info", app.getDiagnostics());
|
|
|
+ }
|
|
|
Assert.assertEquals(FinalApplicationStatus.UNDEFINED,
|
|
|
app.getFinalApplicationStatus());
|
|
|
Assert.assertEquals(YarnApplicationState.FINISHED,
|
|
@@ -126,10 +190,35 @@ public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
|
|
|
@Test
|
|
|
public void testGetApplicationAttemptReport() throws Exception {
|
|
|
- ApplicationAttemptId appAttemptId =
|
|
|
+ final ApplicationAttemptId appAttemptId =
|
|
|
ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1);
|
|
|
- ApplicationAttemptReport appAttempt =
|
|
|
- historyManager.getApplicationAttempt(appAttemptId);
|
|
|
+ ApplicationAttemptReport appAttempt;
|
|
|
+ if (callerUGI == null) {
|
|
|
+ appAttempt = historyManager.getApplicationAttempt(appAttemptId);
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ appAttempt =
|
|
|
+ callerUGI.doAs(new PrivilegedExceptionAction<ApplicationAttemptReport> () {
|
|
|
+ @Override
|
|
|
+ public ApplicationAttemptReport run() throws Exception {
|
|
|
+ return historyManager.getApplicationAttempt(appAttemptId);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
|
|
|
+ // The exception is expected
|
|
|
+ Assert.fail();
|
|
|
+ }
|
|
|
+ } catch (UndeclaredThrowableException 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;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
Assert.assertNotNull(appAttempt);
|
|
|
Assert.assertEquals(appAttemptId, appAttempt.getApplicationAttemptId());
|
|
|
Assert.assertEquals(ContainerId.newInstance(appAttemptId, 1),
|
|
@@ -146,10 +235,36 @@ public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
|
|
|
@Test
|
|
|
public void testGetContainerReport() throws Exception {
|
|
|
- ContainerId containerId =
|
|
|
+ final ContainerId containerId =
|
|
|
ContainerId.newInstance(ApplicationAttemptId.newInstance(
|
|
|
ApplicationId.newInstance(0, 1), 1), 1);
|
|
|
- ContainerReport container = historyManager.getContainer(containerId);
|
|
|
+ ContainerReport container;
|
|
|
+ if (callerUGI == null) {
|
|
|
+ container = historyManager.getContainer(containerId);
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ container =
|
|
|
+ callerUGI.doAs(new PrivilegedExceptionAction<ContainerReport> () {
|
|
|
+ @Override
|
|
|
+ public ContainerReport run() throws Exception {
|
|
|
+ return historyManager.getContainer(containerId);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
|
|
|
+ // The exception is expected
|
|
|
+ Assert.fail();
|
|
|
+ }
|
|
|
+ } catch (UndeclaredThrowableException 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;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
Assert.assertNotNull(container);
|
|
|
Assert.assertEquals(Integer.MAX_VALUE + 1L, container.getCreationTime());
|
|
|
Assert.assertEquals(Integer.MAX_VALUE + 2L, container.getFinishTime());
|
|
@@ -164,7 +279,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
Assert.assertEquals(-1, container.getContainerExitStatus());
|
|
|
Assert.assertEquals("http://0.0.0.0:8188/applicationhistory/logs/" +
|
|
|
"test host:-100/container_0_0001_01_000001/"
|
|
|
- + "container_0_0001_01_000001/test user", container.getLogUrl());
|
|
|
+ + "container_0_0001_01_000001/user1", container.getLogUrl());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -177,29 +292,104 @@ public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
|
|
|
@Test
|
|
|
public void testGetApplicationAttempts() throws Exception {
|
|
|
- Collection<ApplicationAttemptReport> appAttempts =
|
|
|
- historyManager.getApplicationAttempts(ApplicationId.newInstance(0, 1))
|
|
|
- .values();
|
|
|
+ final ApplicationId appId = ApplicationId.newInstance(0, 1);
|
|
|
+ Collection<ApplicationAttemptReport> appAttempts;
|
|
|
+ if (callerUGI == null) {
|
|
|
+ appAttempts = historyManager.getApplicationAttempts(appId).values();
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ appAttempts = callerUGI.doAs(
|
|
|
+ new PrivilegedExceptionAction<Collection<ApplicationAttemptReport>> () {
|
|
|
+ @Override
|
|
|
+ public Collection<ApplicationAttemptReport> run() throws Exception {
|
|
|
+ return historyManager.getApplicationAttempts(appId).values();
|
|
|
+ }
|
|
|
+ });
|
|
|
+ if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
|
|
|
+ // The exception is expected
|
|
|
+ Assert.fail();
|
|
|
+ }
|
|
|
+ } catch (UndeclaredThrowableException 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;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
Assert.assertNotNull(appAttempts);
|
|
|
Assert.assertEquals(SCALE, appAttempts.size());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testGetContainers() throws Exception {
|
|
|
- Collection<ContainerReport> containers =
|
|
|
- historyManager
|
|
|
- .getContainers(
|
|
|
- ApplicationAttemptId.newInstance(
|
|
|
- ApplicationId.newInstance(0, 1), 1)).values();
|
|
|
+ final ApplicationAttemptId appAttemptId =
|
|
|
+ ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1);
|
|
|
+ Collection<ContainerReport> containers;
|
|
|
+ if (callerUGI == null) {
|
|
|
+ containers = historyManager.getContainers(appAttemptId).values();
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ containers = callerUGI.doAs(
|
|
|
+ new PrivilegedExceptionAction<Collection<ContainerReport>> () {
|
|
|
+ @Override
|
|
|
+ public Collection<ContainerReport> run() throws Exception {
|
|
|
+ return historyManager.getContainers(appAttemptId).values();
|
|
|
+ }
|
|
|
+ });
|
|
|
+ if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
|
|
|
+ // The exception is expected
|
|
|
+ Assert.fail();
|
|
|
+ }
|
|
|
+ } catch (UndeclaredThrowableException 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;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
Assert.assertNotNull(containers);
|
|
|
Assert.assertEquals(SCALE, containers.size());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testGetAMContainer() throws Exception {
|
|
|
- ApplicationAttemptId appAttemptId =
|
|
|
+ final ApplicationAttemptId appAttemptId =
|
|
|
ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1);
|
|
|
- ContainerReport container = historyManager.getAMContainer(appAttemptId);
|
|
|
+ ContainerReport container;
|
|
|
+ if (callerUGI == null) {
|
|
|
+ container = historyManager.getAMContainer(appAttemptId);
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ container =
|
|
|
+ callerUGI.doAs(new PrivilegedExceptionAction<ContainerReport> () {
|
|
|
+ @Override
|
|
|
+ public ContainerReport run() throws Exception {
|
|
|
+ return historyManager.getAMContainer(appAttemptId);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ if (callerUGI != null && callerUGI.getShortUserName().equals("user3")) {
|
|
|
+ // The exception is expected
|
|
|
+ Assert.fail();
|
|
|
+ }
|
|
|
+ } catch (UndeclaredThrowableException 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;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
Assert.assertNotNull(container);
|
|
|
Assert.assertEquals(appAttemptId, container.getContainerId()
|
|
|
.getApplicationAttemptId());
|
|
@@ -210,14 +400,18 @@ public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
TimelineEntity entity = new TimelineEntity();
|
|
|
entity.setEntityType(ApplicationMetricsConstants.ENTITY_TYPE);
|
|
|
entity.setEntityId(appId.toString());
|
|
|
+ entity.addPrimaryFilter(
|
|
|
+ TimelineStore.SystemFilter.ENTITY_OWNER.toString(), "yarn");
|
|
|
Map<String, Object> entityInfo = new HashMap<String, Object>();
|
|
|
entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO, "test app");
|
|
|
entityInfo.put(ApplicationMetricsConstants.TYPE_ENTITY_INFO,
|
|
|
"test app type");
|
|
|
- entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, "test user");
|
|
|
+ entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, "user1");
|
|
|
entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, "test queue");
|
|
|
entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO,
|
|
|
Integer.MAX_VALUE + 1L);
|
|
|
+ entityInfo.put(ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO,
|
|
|
+ "user2");
|
|
|
entity.setOtherInfo(entityInfo);
|
|
|
TimelineEvent tEvent = new TimelineEvent();
|
|
|
tEvent.setEventType(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
|
|
@@ -248,6 +442,8 @@ public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
entity.setEntityId(appAttemptId.toString());
|
|
|
entity.addPrimaryFilter(AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER,
|
|
|
appAttemptId.getApplicationId().toString());
|
|
|
+ entity.addPrimaryFilter(
|
|
|
+ TimelineStore.SystemFilter.ENTITY_OWNER.toString(), "yarn");
|
|
|
TimelineEvent tEvent = new TimelineEvent();
|
|
|
tEvent.setEventType(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
|
|
|
tEvent.setTimestamp(Integer.MAX_VALUE + 1L);
|
|
@@ -287,6 +483,8 @@ public class TestApplicationHistoryManagerOnTimelineStore {
|
|
|
entity.setEntityId(containerId.toString());
|
|
|
entity.addPrimaryFilter(ContainerMetricsConstants.PARENT_PRIMARIY_FILTER,
|
|
|
containerId.getApplicationAttemptId().toString());
|
|
|
+ entity.addPrimaryFilter(
|
|
|
+ TimelineStore.SystemFilter.ENTITY_OWNER.toString(), "yarn");
|
|
|
Map<String, Object> entityInfo = new HashMap<String, Object>();
|
|
|
entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO, -1);
|
|
|
entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO, -1);
|