YARN-1214. Register ClientToken MasterKey in SecretManager after it is saved (Jian He via bikas)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1526078 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
524dad1109
commit
929f96ee14
|
@ -77,6 +77,9 @@ Release 2.1.2 - UNRELEASED
|
|||
YARN-1128. FifoPolicy.computeShares throws NPE on empty list of Schedulables
|
||||
(Karthik Kambatla via Sandy Ryza)
|
||||
|
||||
YARN-1214. Register ClientToken MasterKey in SecretManager after it is
|
||||
saved (Jian He via bikas)
|
||||
|
||||
Release 2.1.1-beta - 2013-09-23
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -652,10 +652,18 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
};
|
||||
}
|
||||
|
||||
private static final class RMAppFinishingTransition extends
|
||||
RMAppTransition {
|
||||
private static final class RMAppFinishingTransition extends RMAppTransition {
|
||||
@Override
|
||||
public void transition(RMAppImpl app, RMAppEvent event) {
|
||||
if (event.getType().equals(RMAppEventType.APP_REMOVED)) {
|
||||
RMAppRemovedEvent removeEvent = (RMAppRemovedEvent) event;
|
||||
if (removeEvent.getRemovedException() != null) {
|
||||
LOG.error(
|
||||
"Failed to remove application: " + removeEvent.getApplicationId(),
|
||||
removeEvent.getRemovedException());
|
||||
ExitUtil.terminate(1, removeEvent.getRemovedException());
|
||||
}
|
||||
}
|
||||
app.finishTime = System.currentTimeMillis();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Set;
|
|||
|
||||
import javax.crypto.SecretKey;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
||||
|
@ -151,9 +152,12 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
|
|||
Token<AMRMTokenIdentifier> getAMRMToken();
|
||||
|
||||
/**
|
||||
* The master key for client-to-AM tokens for this app attempt
|
||||
* The master key for client-to-AM tokens for this app attempt. This is only
|
||||
* used for RMStateStore. Normal operation must invoke the secret manager to
|
||||
* get the key and not use the local key directly.
|
||||
* @return The master key for client-to-AM tokens for this app attempt
|
||||
*/
|
||||
@LimitedPrivate("RMStateStore")
|
||||
SecretKey getClientTokenMasterKey();
|
||||
|
||||
/**
|
||||
|
|
|
@ -501,6 +501,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|||
proxiedTrackingUrl = origTrackingUrl;
|
||||
}
|
||||
|
||||
// This is only used for RMStateStore. Normal operation must invoke the secret
|
||||
// manager to get the key and not use the local key directly.
|
||||
@Override
|
||||
public SecretKey getClientTokenMasterKey() {
|
||||
return this.clientTokenMasterKey;
|
||||
|
@ -734,9 +736,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|||
.registerAppAttempt(appAttempt.applicationAttemptId);
|
||||
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
appAttempt.clientTokenMasterKey = appAttempt.rmContext
|
||||
.getClientToAMTokenSecretManager()
|
||||
.registerApplication(appAttempt.applicationAttemptId);
|
||||
appAttempt.clientTokenMasterKey =
|
||||
appAttempt.rmContext.getClientToAMTokenSecretManager()
|
||||
.createMasterKey(appAttempt.applicationAttemptId);
|
||||
}
|
||||
|
||||
// create AMRMToken
|
||||
|
@ -922,6 +924,12 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|||
RMAppAttemptEvent event) {
|
||||
// Register with AMLivelinessMonitor
|
||||
appAttempt.attemptLaunched();
|
||||
|
||||
// register the ClientTokenMasterKey after it is saved in the store,
|
||||
// otherwise client may hold an invalid ClientToken after RM restarts.
|
||||
appAttempt.rmContext.getClientToAMTokenSecretManager()
|
||||
.registerApplication(appAttempt.getAppAttemptId(),
|
||||
appAttempt.getClientTokenMasterKey());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -33,17 +33,21 @@ public class ClientToAMTokenSecretManagerInRM extends
|
|||
private Map<ApplicationAttemptId, SecretKey> masterKeys =
|
||||
new HashMap<ApplicationAttemptId, SecretKey>();
|
||||
|
||||
public synchronized SecretKey registerApplication(
|
||||
public synchronized SecretKey createMasterKey(
|
||||
ApplicationAttemptId applicationAttemptID) {
|
||||
SecretKey key = generateSecret();
|
||||
this.masterKeys.put(applicationAttemptID, key);
|
||||
return key;
|
||||
return generateSecret();
|
||||
}
|
||||
|
||||
public synchronized void registerApplication(
|
||||
ApplicationAttemptId applicationAttemptID, SecretKey key) {
|
||||
this.masterKeys.put(applicationAttemptID, key);
|
||||
}
|
||||
|
||||
// Only for RM recovery
|
||||
public synchronized SecretKey registerMasterKey(
|
||||
ApplicationAttemptId applicationAttemptID, byte[] keyData) {
|
||||
SecretKey key = createSecretKey(keyData);
|
||||
this.masterKeys.put(applicationAttemptID, key);
|
||||
registerApplication(applicationAttemptID, key);
|
||||
return key;
|
||||
}
|
||||
|
||||
|
|
|
@ -289,7 +289,7 @@ public class TestRMStateStore extends ClientBaseWithFixes{
|
|||
HashSet<Token<?>> attemptTokenSet1 = new HashSet<Token<?>>();
|
||||
attemptTokenSet1.add(appAttemptToken1);
|
||||
SecretKey clientTokenKey1 =
|
||||
clientToAMTokenMgr.registerApplication(attemptId1);
|
||||
clientToAMTokenMgr.createMasterKey(attemptId1);
|
||||
|
||||
ContainerId containerId1 = storeAttempt(store, attemptId1,
|
||||
"container_1352994193343_0001_01_000001",
|
||||
|
@ -305,7 +305,7 @@ public class TestRMStateStore extends ClientBaseWithFixes{
|
|||
HashSet<Token<?>> attemptTokenSet2 = new HashSet<Token<?>>();
|
||||
attemptTokenSet2.add(appAttemptToken2);
|
||||
SecretKey clientTokenKey2 =
|
||||
clientToAMTokenMgr.registerApplication(attemptId2);
|
||||
clientToAMTokenMgr.createMasterKey(attemptId2);
|
||||
|
||||
ContainerId containerId2 = storeAttempt(store, attemptId2,
|
||||
"container_1352994193343_0001_02_000001",
|
||||
|
|
|
@ -18,13 +18,18 @@
|
|||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
|
||||
|
||||
import static org.mockito.Mockito.mock;
|
||||
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.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;
|
||||
|
||||
|
@ -35,10 +40,13 @@ 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;
|
||||
|
@ -54,7 +62,10 @@ 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.RMAppAttemptStoredEvent;
|
||||
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;
|
||||
|
@ -62,7 +73,7 @@ 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.junit.After;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
@ -81,6 +92,7 @@ public class TestRMAppTransitions {
|
|||
private static int appId = 1;
|
||||
private DrainDispatcher rmDispatcher;
|
||||
private RMStateStore store;
|
||||
private YarnScheduler scheduler;
|
||||
|
||||
// ignore all the RM application attempt events
|
||||
private static final class TestApplicationAttemptEventDispatcher implements
|
||||
|
@ -206,7 +218,8 @@ public class TestRMAppTransitions {
|
|||
String queue = MockApps.newQueue();
|
||||
// ensure max application attempts set to known value
|
||||
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, maxAppAttempts);
|
||||
YarnScheduler scheduler = mock(YarnScheduler.class);
|
||||
scheduler = mock(YarnScheduler.class);
|
||||
|
||||
ApplicationMasterService masterService =
|
||||
new ApplicationMasterService(rmContext, scheduler);
|
||||
|
||||
|
@ -392,8 +405,7 @@ public class TestRMAppTransitions {
|
|||
RMApp application = testCreateAppRemoving(submissionContext);
|
||||
// REMOVING => FINISHING event RMAppEventType.APP_REMOVED
|
||||
RMAppEvent finishingEvent =
|
||||
new RMAppEvent(application.getApplicationId(),
|
||||
RMAppEventType.APP_REMOVED);
|
||||
new RMAppRemovedEvent(application.getApplicationId(), null);
|
||||
application.handle(finishingEvent);
|
||||
assertAppState(RMAppState.FINISHING, application);
|
||||
assertTimesAtFinish(application);
|
||||
|
@ -816,6 +828,14 @@ public class TestRMAppTransitions {
|
|||
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);
|
||||
|
@ -830,4 +850,33 @@ public class TestRMAppTransitions {
|
|||
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 RMAppAttemptStoredEvent(attempt.getAppAttemptId(), null));
|
||||
attempt.handle(new RMAppAttemptEvent(attempt.getAppAttemptId(),
|
||||
RMAppAttemptEventType.LAUNCHED));
|
||||
|
||||
assertEquals(RMAppAttemptState.LAUNCHED, attempt.getAppAttemptState());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -294,9 +294,11 @@ public class TestRMAppAttemptTransitions {
|
|||
assertEquals(0, applicationAttempt.getRanNodes().size());
|
||||
assertNull(applicationAttempt.getFinalApplicationStatus());
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
verify(clientToAMTokenManager).registerApplication(
|
||||
verify(clientToAMTokenManager).createMasterKey(
|
||||
applicationAttempt.getAppAttemptId());
|
||||
assertNotNull(applicationAttempt.createClientToken("some client"));
|
||||
// can't create ClientToken as at this time ClientTokenMasterKey has
|
||||
// not been registered in the SecretManager
|
||||
assertNull(applicationAttempt.createClientToken("some client"));
|
||||
}
|
||||
assertNull(applicationAttempt.createClientToken(null));
|
||||
assertNotNull(applicationAttempt.getAMRMToken());
|
||||
|
@ -428,7 +430,11 @@ public class TestRMAppAttemptTransitions {
|
|||
assertEquals(RMAppAttemptState.LAUNCHED,
|
||||
applicationAttempt.getAppAttemptState());
|
||||
assertEquals(container, applicationAttempt.getMasterContainer());
|
||||
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
// ClientTokenMasterKey has been registered in SecretManager, it's able to
|
||||
// create ClientToken now
|
||||
assertNotNull(applicationAttempt.createClientToken("some client"));
|
||||
}
|
||||
// TODO - need to add more checks relevant to this state
|
||||
}
|
||||
|
||||
|
@ -561,6 +567,11 @@ public class TestRMAppAttemptTransitions {
|
|||
}
|
||||
|
||||
private void launchApplicationAttempt(Container container) {
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
// Before LAUNCHED state, can't create ClientToken as at this time
|
||||
// ClientTokenMasterKey has not been registered in the SecretManager
|
||||
assertNull(applicationAttempt.createClientToken("some client"));
|
||||
}
|
||||
applicationAttempt.handle(
|
||||
new RMAppAttemptEvent(applicationAttempt.getAppAttemptId(),
|
||||
RMAppAttemptEventType.LAUNCHED));
|
||||
|
|
|
@ -194,16 +194,6 @@ public class TestClientToAMTokens {
|
|||
nm1.nodeHeartbeat(true);
|
||||
dispatcher.await();
|
||||
|
||||
// Get the app-report.
|
||||
GetApplicationReportRequest request =
|
||||
Records.newRecord(GetApplicationReportRequest.class);
|
||||
request.setApplicationId(app.getApplicationId());
|
||||
GetApplicationReportResponse reportResponse =
|
||||
rm.getClientRMService().getApplicationReport(request);
|
||||
ApplicationReport appReport = reportResponse.getApplicationReport();
|
||||
org.apache.hadoop.yarn.api.records.Token originalClientToAMToken =
|
||||
appReport.getClientToAMToken();
|
||||
|
||||
ApplicationAttemptId appAttempt = app.getCurrentAppAttempt().getAppAttemptId();
|
||||
final MockAM mockAM =
|
||||
new MockAM(rm.getRMContext(), rm.getApplicationMasterService(),
|
||||
|
@ -224,7 +214,17 @@ public class TestClientToAMTokens {
|
|||
return response;
|
||||
}
|
||||
});
|
||||
|
||||
|
||||
// Get the app-report.
|
||||
GetApplicationReportRequest request =
|
||||
Records.newRecord(GetApplicationReportRequest.class);
|
||||
request.setApplicationId(app.getApplicationId());
|
||||
GetApplicationReportResponse reportResponse =
|
||||
rm.getClientRMService().getApplicationReport(request);
|
||||
ApplicationReport appReport = reportResponse.getApplicationReport();
|
||||
org.apache.hadoop.yarn.api.records.Token originalClientToAMToken =
|
||||
appReport.getClientToAMToken();
|
||||
|
||||
// ClientToAMToken master key should have been received on register
|
||||
// application master response.
|
||||
Assert.assertNotNull(response.getClientToAMTokenMasterKey());
|
||||
|
|
Loading…
Reference in New Issue