|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
|
|
|
|
|
|
+import static org.junit.Assert.assertArrayEquals;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
@@ -25,12 +26,12 @@ import static org.junit.Assert.fail;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
-import java.util.ArrayList;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
-import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
+import javax.crypto.SecretKey;
|
|
|
+
|
|
|
import junit.framework.Assert;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -55,7 +56,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
|
|
-import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
|
|
@@ -198,7 +198,7 @@ public class TestRMStateStore {
|
|
|
|
|
|
ContainerId storeAttempt(RMStateStore store, ApplicationAttemptId attemptId,
|
|
|
String containerIdStr, Token<AMRMTokenIdentifier> appToken,
|
|
|
- Token<ClientToAMTokenIdentifier> clientToAMToken, TestDispatcher dispatcher)
|
|
|
+ SecretKey clientTokenMasterKey, TestDispatcher dispatcher)
|
|
|
throws Exception {
|
|
|
|
|
|
Container container = new ContainerPBImpl();
|
|
@@ -207,7 +207,8 @@ public class TestRMStateStore {
|
|
|
when(mockAttempt.getAppAttemptId()).thenReturn(attemptId);
|
|
|
when(mockAttempt.getMasterContainer()).thenReturn(container);
|
|
|
when(mockAttempt.getAMRMToken()).thenReturn(appToken);
|
|
|
- when(mockAttempt.getClientToAMToken()).thenReturn(clientToAMToken);
|
|
|
+ when(mockAttempt.getClientTokenMasterKey())
|
|
|
+ .thenReturn(clientTokenMasterKey);
|
|
|
dispatcher.attemptId = attemptId;
|
|
|
dispatcher.storedException = null;
|
|
|
store.storeApplicationAttempt(mockAttempt);
|
|
@@ -215,7 +216,6 @@ public class TestRMStateStore {
|
|
|
return container.getId();
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
void testRMAppStateStore(RMStateStoreHelper stateStoreHelper) throws Exception {
|
|
|
long submitTime = System.currentTimeMillis();
|
|
|
Configuration conf = new YarnConfiguration();
|
|
@@ -233,33 +233,33 @@ public class TestRMStateStore {
|
|
|
ApplicationId appId1 = attemptId1.getApplicationId();
|
|
|
storeApp(store, appId1, submitTime);
|
|
|
|
|
|
- // create application token1 for attempt1
|
|
|
- List<Token<?>> appAttemptToken1 =
|
|
|
- generateTokens(attemptId1, appTokenMgr, clientToAMTokenMgr, conf);
|
|
|
+ // create application token and client token key for attempt1
|
|
|
+ Token<AMRMTokenIdentifier> appAttemptToken1 =
|
|
|
+ generateAMRMToken(attemptId1, appTokenMgr);
|
|
|
HashSet<Token<?>> attemptTokenSet1 = new HashSet<Token<?>>();
|
|
|
- attemptTokenSet1.addAll(appAttemptToken1);
|
|
|
+ attemptTokenSet1.add(appAttemptToken1);
|
|
|
+ SecretKey clientTokenKey1 =
|
|
|
+ clientToAMTokenMgr.registerApplication(attemptId1);
|
|
|
|
|
|
ContainerId containerId1 = storeAttempt(store, attemptId1,
|
|
|
"container_1352994193343_0001_01_000001",
|
|
|
- (Token<AMRMTokenIdentifier>) (appAttemptToken1.get(0)),
|
|
|
- (Token<ClientToAMTokenIdentifier>)(appAttemptToken1.get(1)),
|
|
|
- dispatcher);
|
|
|
+ appAttemptToken1, clientTokenKey1, dispatcher);
|
|
|
|
|
|
String appAttemptIdStr2 = "appattempt_1352994193343_0001_000002";
|
|
|
ApplicationAttemptId attemptId2 =
|
|
|
ConverterUtils.toApplicationAttemptId(appAttemptIdStr2);
|
|
|
|
|
|
- // create application token2 for attempt2
|
|
|
- List<Token<?>> appAttemptToken2 =
|
|
|
- generateTokens(attemptId2, appTokenMgr, clientToAMTokenMgr, conf);
|
|
|
+ // create application token and client token key for attempt2
|
|
|
+ Token<AMRMTokenIdentifier> appAttemptToken2 =
|
|
|
+ generateAMRMToken(attemptId2, appTokenMgr);
|
|
|
HashSet<Token<?>> attemptTokenSet2 = new HashSet<Token<?>>();
|
|
|
- attemptTokenSet2.addAll(appAttemptToken2);
|
|
|
+ attemptTokenSet2.add(appAttemptToken2);
|
|
|
+ SecretKey clientTokenKey2 =
|
|
|
+ clientToAMTokenMgr.registerApplication(attemptId2);
|
|
|
|
|
|
ContainerId containerId2 = storeAttempt(store, attemptId2,
|
|
|
"container_1352994193343_0001_02_000001",
|
|
|
- (Token<AMRMTokenIdentifier>) (appAttemptToken2.get(0)),
|
|
|
- (Token<ClientToAMTokenIdentifier>)(appAttemptToken2.get(1)),
|
|
|
- dispatcher);
|
|
|
+ appAttemptToken2, clientTokenKey2, dispatcher);
|
|
|
|
|
|
ApplicationAttemptId attemptIdRemoved = ConverterUtils
|
|
|
.toApplicationAttemptId("appattempt_1352994193343_0002_000001");
|
|
@@ -306,8 +306,12 @@ public class TestRMStateStore {
|
|
|
assertEquals(containerId1, attemptState.getMasterContainer().getId());
|
|
|
// attempt1 applicationToken is loaded correctly
|
|
|
HashSet<Token<?>> savedTokens = new HashSet<Token<?>>();
|
|
|
- savedTokens.addAll(attemptState.getAppAttemptTokens().getAllTokens());
|
|
|
+ savedTokens.addAll(attemptState.getAppAttemptCredentials().getAllTokens());
|
|
|
assertEquals(attemptTokenSet1, savedTokens);
|
|
|
+ // attempt1 client token master key is loaded correctly
|
|
|
+ assertArrayEquals(clientTokenKey1.getEncoded(),
|
|
|
+ attemptState.getAppAttemptCredentials()
|
|
|
+ .getSecretKey(RMStateStore.AM_CLIENT_TOKEN_MASTER_KEY_NAME));
|
|
|
|
|
|
attemptState = appState.getAttempt(attemptId2);
|
|
|
// attempt2 is loaded correctly
|
|
@@ -317,8 +321,12 @@ public class TestRMStateStore {
|
|
|
assertEquals(containerId2, attemptState.getMasterContainer().getId());
|
|
|
// attempt2 applicationToken is loaded correctly
|
|
|
savedTokens.clear();
|
|
|
- savedTokens.addAll(attemptState.getAppAttemptTokens().getAllTokens());
|
|
|
+ savedTokens.addAll(attemptState.getAppAttemptCredentials().getAllTokens());
|
|
|
assertEquals(attemptTokenSet2, savedTokens);
|
|
|
+ // attempt2 client token master key is loaded correctly
|
|
|
+ assertArrayEquals(clientTokenKey2.getEncoded(),
|
|
|
+ attemptState.getAppAttemptCredentials()
|
|
|
+ .getSecretKey(RMStateStore.AM_CLIENT_TOKEN_MASTER_KEY_NAME));
|
|
|
|
|
|
// assert store is in expected state after everything is cleaned
|
|
|
assertTrue(stateStoreHelper.isFinalStateValid());
|
|
@@ -357,24 +365,14 @@ public class TestRMStateStore {
|
|
|
Assert.assertEquals(sequenceNumber, secretManagerState.getDTSequenceNumber());
|
|
|
}
|
|
|
|
|
|
- private List<Token<?>> generateTokens(ApplicationAttemptId attemptId,
|
|
|
- AMRMTokenSecretManager appTokenMgr,
|
|
|
- ClientToAMTokenSecretManagerInRM clientToAMTokenMgr, Configuration conf) {
|
|
|
+ private Token<AMRMTokenIdentifier> generateAMRMToken(
|
|
|
+ ApplicationAttemptId attemptId,
|
|
|
+ AMRMTokenSecretManager appTokenMgr) {
|
|
|
AMRMTokenIdentifier appTokenId =
|
|
|
new AMRMTokenIdentifier(attemptId);
|
|
|
Token<AMRMTokenIdentifier> appToken =
|
|
|
new Token<AMRMTokenIdentifier>(appTokenId, appTokenMgr);
|
|
|
appToken.setService(new Text("appToken service"));
|
|
|
-
|
|
|
- ClientToAMTokenIdentifier clientToAMTokenId =
|
|
|
- new ClientToAMTokenIdentifier(attemptId);
|
|
|
- clientToAMTokenMgr.registerApplication(attemptId);
|
|
|
- Token<ClientToAMTokenIdentifier> clientToAMToken =
|
|
|
- new Token<ClientToAMTokenIdentifier>(clientToAMTokenId, clientToAMTokenMgr);
|
|
|
- clientToAMToken.setService(new Text("clientToAMToken service"));
|
|
|
- List<Token<?>> tokenPair = new ArrayList<Token<?>>();
|
|
|
- tokenPair.add(0, appToken);
|
|
|
- tokenPair.add(1, clientToAMToken);
|
|
|
- return tokenPair;
|
|
|
+ return appToken;
|
|
|
}
|
|
|
}
|