YARN-1416. Fixed a few invalid transitions in RMApp, RMAppAttempt and in some tests. Contributed by Jian He.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1545448 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2013-11-25 23:10:34 +00:00
parent b1d6574a11
commit 6c243fceac
5 changed files with 56 additions and 94 deletions

View File

@ -176,6 +176,9 @@ Release 2.3.0 - UNRELEASED
YARN-1320. Fixed Distributed Shell application to respect custom log4j
properties file. (Xuan Gong via vinodkv)
YARN-1416. Fixed a few invalid transitions in RMApp, RMAppAttempt and in some
tests. (Jian He via vinodkv)
Release 2.2.1 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -130,7 +130,7 @@ public class RMAppImpl implements RMApp, Recoverable {
.addTransition(RMAppState.NEW, RMAppState.NEW,
RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
.addTransition(RMAppState.NEW, RMAppState.NEW_SAVING,
RMAppEventType.START, new RMAppSavingTransition())
RMAppEventType.START, new RMAppNewlySavingTransition())
.addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED,
RMAppState.RUNNING, RMAppState.FINISHED, RMAppState.FAILED,
RMAppState.KILLED, RMAppState.FINAL_SAVING),
@ -215,7 +215,8 @@ public class RMAppImpl implements RMApp, Recoverable {
new AttemptFinishedAtFinalSavingTransition())
// ignorable transitions
.addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
EnumSet.of(RMAppEventType.NODE_UPDATE, RMAppEventType.KILL))
EnumSet.of(RMAppEventType.NODE_UPDATE, RMAppEventType.KILL,
RMAppEventType.APP_NEW_SAVED))
// Transitions from FINISHING state
.addTransition(RMAppState.FINISHING, RMAppState.FINISHED,
@ -760,7 +761,7 @@ public class RMAppImpl implements RMApp, Recoverable {
return msg;
}
private static final class RMAppSavingTransition extends RMAppTransition {
private static final class RMAppNewlySavingTransition extends RMAppTransition {
@Override
public void transition(RMAppImpl app, RMAppEvent event) {

View File

@ -334,6 +334,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
// Saving in scheduler
RMAppAttemptEventType.CONTAINER_ALLOCATED,
RMAppAttemptEventType.CONTAINER_ACQUIRED,
RMAppAttemptEventType.ATTEMPT_NEW_SAVED,
RMAppAttemptEventType.KILL))
// Transitions from FAILED State

View File

@ -18,19 +18,14 @@
package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
import static org.junit.Assert.assertEquals;
import static org.junit.Assume.assumeTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import junit.framework.Assert;
@ -41,13 +36,10 @@ import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.DrainDispatcher;
@ -64,11 +56,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@ -76,7 +65,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -290,7 +278,6 @@ public class TestRMAppTransitions {
// test to make sure times are set when app finishes
private void assertTimesAtFinish(RMApp application) {
sendAppUpdateSavedEvent(application);
assertStartTimeSet(application);
Assert.assertTrue("application finish time is not greater then 0",
(application.getFinishTime() > 0));
@ -303,7 +290,6 @@ public class TestRMAppTransitions {
}
private void assertKilled(RMApp application) {
sendAppUpdateSavedEvent(application);
assertTimesAtFinish(application);
assertAppState(RMAppState.KILLED, application);
assertFinalAppStatus(FinalApplicationStatus.KILLED, application);
@ -314,6 +300,7 @@ public class TestRMAppTransitions {
private void assertAppAndAttemptKilled(RMApp application)
throws InterruptedException {
sendAppUpdateSavedEvent(application);
assertKilled(application);
// send attempt final state saved event.
application.getCurrentAppAttempt().handle(
@ -325,7 +312,6 @@ public class TestRMAppTransitions {
}
private void assertFailed(RMApp application, String regex) {
sendAppUpdateSavedEvent(application);
assertTimesAtFinish(application);
assertAppState(RMAppState.FAILED, application);
assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
@ -475,6 +461,7 @@ public class TestRMAppTransitions {
rmDispatcher.await();
RMAppAttempt appAttempt = application.getCurrentAppAttempt();
Assert.assertEquals(1, appAttempt.getAppAttemptId().getAttemptId());
sendAppUpdateSavedEvent(application);
assertFailed(application,
".*Unmanaged application.*Failing the application.*");
}
@ -504,6 +491,7 @@ public class TestRMAppTransitions {
new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertKilled(application);
}
@ -518,6 +506,7 @@ public class TestRMAppTransitions {
new RMAppRejectedEvent(application.getApplicationId(), rejectedText);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertFailed(application, rejectedText);
}
@ -531,6 +520,7 @@ public class TestRMAppTransitions {
new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertKilled(application);
}
@ -545,6 +535,7 @@ public class TestRMAppTransitions {
new RMAppRejectedEvent(application.getApplicationId(), rejectedText);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertFailed(application, rejectedText);
}
@ -559,6 +550,7 @@ public class TestRMAppTransitions {
new RMAppRejectedEvent(application.getApplicationId(), rejectedText);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertFailed(application, rejectedText);
}
@ -603,6 +595,7 @@ public class TestRMAppTransitions {
RMAppEventType.ATTEMPT_FAILED, message);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertFailed(application, ".*" + message + ".*Failing the application.*");
}
@ -635,7 +628,7 @@ public class TestRMAppTransitions {
new RMAppFinishedAttemptEvent(application.getApplicationId(), null);
application.handle(finishEvent);
assertAppState(RMAppState.FINAL_SAVING, application);
sendAppUpdateSavedEvent(application);
assertKilled(application);
}
@ -681,6 +674,7 @@ public class TestRMAppTransitions {
RMAppEventType.ATTEMPT_FAILED, "");
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertFailed(application, ".*Failing the application.*");
// FAILED => FAILED event RMAppEventType.KILL
@ -758,6 +752,7 @@ public class TestRMAppTransitions {
new RMAppRejectedEvent(application.getApplicationId(), "");
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertTimesAtFinish(application);
assertAppState(RMAppState.FAILED, application);
@ -769,10 +764,6 @@ public class TestRMAppTransitions {
assertTimesAtFinish(application);
assertAppState(RMAppState.FAILED, application);
// FAILED => FAILED event RMAppEventType.APP_SAVED
event = new RMAppNewSavedEvent(application.getApplicationId(), null);
application.handle(event);
rmDispatcher.await();
assertTimesAtFinish(application);
assertAppState(RMAppState.FAILED, application);
}
@ -788,6 +779,7 @@ public class TestRMAppTransitions {
new RMAppEvent(application.getApplicationId(), RMAppEventType.KILL);
application.handle(event);
rmDispatcher.await();
sendAppUpdateSavedEvent(application);
assertTimesAtFinish(application);
assertAppState(RMAppState.KILLED, application);
@ -824,10 +816,6 @@ public class TestRMAppTransitions {
assertTimesAtFinish(application);
assertAppState(RMAppState.KILLED, application);
// KILLED => KILLED event RMAppEventType.APP_SAVED
event = new RMAppNewSavedEvent(application.getApplicationId(), null);
application.handle(event);
rmDispatcher.await();
assertTimesAtFinish(application);
assertAppState(RMAppState.KILLED, application);
}
@ -841,70 +829,4 @@ public class TestRMAppTransitions {
report = app.createAndGetApplicationReport("clientuser", true);
Assert.assertNotNull(report.getApplicationResourceUsageReport());
}
@Test
public void testClientTokens() throws Exception {
assumeTrue(isSecurityEnabled);
RMApp app = createNewTestApp(null);
assertAppState(RMAppState.NEW, app);
ApplicationReport report = app.createAndGetApplicationReport(null, true);
Assert.assertNull(report.getClientToAMToken());
report = app.createAndGetApplicationReport("clientuser", true);
Assert.assertNull(report.getClientToAMToken());
app = testCreateAppRunning(null);
rmDispatcher.await();
assertAppState(RMAppState.RUNNING, app);
report = app.createAndGetApplicationReport("clientuser", true);
Assert.assertNull(report.getClientToAMToken());
// this method is to make AMLaunchedTransition invoked inside which
// ClientTokenMasterKey is registered in ClientTokenSecretManager
moveCurrentAttemptToLaunchedState(app.getCurrentAppAttempt());
report = app.createAndGetApplicationReport(null, true);
Assert.assertNull(report.getClientToAMToken());
report = app.createAndGetApplicationReport("clientuser", true);
Assert.assertNotNull(report.getClientToAMToken());
// kill the app attempt and verify client token is unavailable
app.handle(new RMAppEvent(app.getApplicationId(), RMAppEventType.KILL));
rmDispatcher.await();
assertAppAndAttemptKilled(app);
report = app.createAndGetApplicationReport(null, true);
Assert.assertNull(report.getClientToAMToken());
report = app.createAndGetApplicationReport("clientuser", true);
Assert.assertNull(report.getClientToAMToken());
}
@SuppressWarnings("unchecked")
private void moveCurrentAttemptToLaunchedState(RMAppAttempt attempt) {
attempt.handle(new RMAppAttemptEvent(attempt.getAppAttemptId(),
RMAppAttemptEventType.APP_ACCEPTED));
// Mock the allocation of AM container
Container container = mock(Container.class);
Resource resource = BuilderUtils.newResource(2048, 1);
when(container.getId()).thenReturn(
BuilderUtils.newContainerId(attempt.getAppAttemptId(), 1));
when(container.getResource()).thenReturn(resource);
Allocation allocation = mock(Allocation.class);
when(allocation.getContainers()).thenReturn(
Collections.singletonList(container));
when(allocation.getContainers()).
thenReturn(Collections.singletonList(container));
when(
scheduler.allocate(any(ApplicationAttemptId.class), any(List.class),
any(List.class), any(List.class), any(List.class))).thenReturn(
allocation);
attempt.handle(new RMAppAttemptContainerAllocatedEvent(attempt
.getAppAttemptId(), container));
attempt
.handle(new RMAppAttemptNewSavedEvent(attempt.getAppAttemptId(), null));
attempt.handle(new RMAppAttemptEvent(attempt.getAppAttemptId(),
RMAppAttemptEventType.LAUNCHED));
assertEquals(RMAppAttemptState.LAUNCHED, attempt.getAppAttemptState());
}
}

View File

@ -23,6 +23,7 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assume.assumeTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
@ -44,6 +45,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -57,6 +59,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.event.InlineDispatcher;
import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
@ -74,9 +77,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
@ -1068,6 +1071,38 @@ public class TestRMAppAttemptTransitions {
diagnostics, 0, false);
}
// this is to test user can get client tokens only after the client token
// master key is saved in the state store and also registered in
// ClientTokenSecretManager
@Test
public void testGetClientToken() throws Exception {
assumeTrue(isSecurityEnabled);
Container amContainer = allocateApplicationAttempt();
// before attempt is launched, can not get ClientToken
Token<ClientToAMTokenIdentifier> token =
applicationAttempt.createClientToken(null);
Assert.assertNull(token);
token = applicationAttempt.createClientToken("clientuser");
Assert.assertNull(token);
launchApplicationAttempt(amContainer);
// after attempt is launched , can get ClientToken
token = applicationAttempt.createClientToken(null);
Assert.assertNull(token);
token = applicationAttempt.createClientToken("clientuser");
Assert.assertNotNull(token);
applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
.getAppAttemptId(), RMAppAttemptEventType.KILL));
sendAttemptUpdateSavedEvent(applicationAttempt);
// after attempt is killed, can not get Client Token
token = applicationAttempt.createClientToken(null);
Assert.assertNull(token);
token = applicationAttempt.createClientToken("clientuser");
Assert.assertNull(token);
}
private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) {
verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId);
if (UserGroupInformation.isSecurityEnabled()) {