|
@@ -35,6 +35,7 @@ import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Collection;
|
|
|
import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.concurrent.BlockingQueue;
|
|
@@ -80,7 +81,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.event.Event;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SystemCredentialsForAppsProto;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|
|
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeHeartbeatResponsePBImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
@@ -95,12 +98,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.DelegationTokenToRenew;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
+import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
@@ -146,8 +151,13 @@ public class TestDelegationTokenRenewer {
|
|
|
@Override
|
|
|
public long renew(Token<?> t, Configuration conf) throws IOException {
|
|
|
if ( !(t instanceof MyToken)) {
|
|
|
- // renew in 3 seconds
|
|
|
- return System.currentTimeMillis() + 3000;
|
|
|
+ if(conf.get("override_token_expire_time") != null) {
|
|
|
+ return System.currentTimeMillis() +
|
|
|
+ Long.parseLong(conf.get("override_token_expire_time"));
|
|
|
+ } else {
|
|
|
+ // renew in 3 seconds
|
|
|
+ return System.currentTimeMillis() + 3000;
|
|
|
+ }
|
|
|
}
|
|
|
MyToken token = (MyToken)t;
|
|
|
if(token.isCanceled()) {
|
|
@@ -201,6 +211,7 @@ public class TestDelegationTokenRenewer {
|
|
|
counter = new AtomicInteger(0);
|
|
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
"kerberos");
|
|
|
+ conf.set("override_token_expire_time", "3000");
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
eventQueue = new LinkedBlockingQueue<Event>();
|
|
|
dispatcher = new AsyncDispatcher(eventQueue);
|
|
@@ -209,7 +220,7 @@ public class TestDelegationTokenRenewer {
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
when(mockContext.getSystemCredentialsForApps()).thenReturn(
|
|
|
- new ConcurrentHashMap<ApplicationId, ByteBuffer>());
|
|
|
+ new ConcurrentHashMap<ApplicationId, SystemCredentialsForAppsProto>());
|
|
|
when(mockContext.getDelegationTokenRenewer()).thenReturn(
|
|
|
delegationTokenRenewer);
|
|
|
when(mockContext.getDispatcher()).thenReturn(dispatcher);
|
|
@@ -581,7 +592,7 @@ public class TestDelegationTokenRenewer {
|
|
|
createNewDelegationTokenRenewer(lconf, counter);
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
when(mockContext.getSystemCredentialsForApps()).thenReturn(
|
|
|
- new ConcurrentHashMap<ApplicationId, ByteBuffer>());
|
|
|
+ new ConcurrentHashMap<ApplicationId, SystemCredentialsForAppsProto>());
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
when(mockContext.getDelegationTokenRenewer()).thenReturn(
|
|
@@ -661,7 +672,7 @@ public class TestDelegationTokenRenewer {
|
|
|
createNewDelegationTokenRenewer(conf, counter);
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
when(mockContext.getSystemCredentialsForApps()).thenReturn(
|
|
|
- new ConcurrentHashMap<ApplicationId, ByteBuffer>());
|
|
|
+ new ConcurrentHashMap<ApplicationId, SystemCredentialsForAppsProto>());
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
when(mockContext.getDelegationTokenRenewer()).thenReturn(
|
|
@@ -766,7 +777,7 @@ public class TestDelegationTokenRenewer {
|
|
|
createNewDelegationTokenRenewer(conf, counter);
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
when(mockContext.getSystemCredentialsForApps()).thenReturn(
|
|
|
- new ConcurrentHashMap<ApplicationId, ByteBuffer>());
|
|
|
+ new ConcurrentHashMap<ApplicationId, SystemCredentialsForAppsProto>());
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
InetSocketAddress sockAddr =
|
|
@@ -825,7 +836,7 @@ public class TestDelegationTokenRenewer {
|
|
|
createNewDelegationTokenRenewer(conf, counter);
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
when(mockContext.getSystemCredentialsForApps()).thenReturn(
|
|
|
- new ConcurrentHashMap<ApplicationId, ByteBuffer>());
|
|
|
+ new ConcurrentHashMap<ApplicationId, SystemCredentialsForAppsProto>());
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
InetSocketAddress sockAddr =
|
|
@@ -890,7 +901,7 @@ public class TestDelegationTokenRenewer {
|
|
|
}
|
|
|
|
|
|
|
|
|
- @Test (timeout = 20000)
|
|
|
+ @Test(timeout = 30000)
|
|
|
public void testReplaceExpiringDelegationToken() throws Exception {
|
|
|
conf.setBoolean(YarnConfiguration.RM_PROXY_USER_PRIVILEGES_ENABLED, true);
|
|
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
@@ -969,8 +980,14 @@ public class TestDelegationTokenRenewer {
|
|
|
new MockNM("127.0.0.1:1234", 15120, rm.getResourceTrackerService());
|
|
|
nm1.registerNode();
|
|
|
NodeHeartbeatResponse response = nm1.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ NodeHeartbeatResponse proto = new NodeHeartbeatResponsePBImpl(
|
|
|
+ ((NodeHeartbeatResponsePBImpl) response).getProto());
|
|
|
+
|
|
|
ByteBuffer tokenBuffer =
|
|
|
- response.getSystemCredentialsForApps().get(app.getApplicationId());
|
|
|
+ YarnServerBuilderUtils
|
|
|
+ .convertFromProtoFormat(proto.getSystemCredentialsForApps())
|
|
|
+ .get(app.getApplicationId());
|
|
|
Assert.assertNotNull(tokenBuffer);
|
|
|
Credentials appCredentials = new Credentials();
|
|
|
DataInputByteBuffer buf = new DataInputByteBuffer();
|
|
@@ -1062,8 +1079,14 @@ public class TestDelegationTokenRenewer {
|
|
|
new MockNM("127.0.0.1:1234", 15120, rm2.getResourceTrackerService());
|
|
|
nm1.registerNode();
|
|
|
NodeHeartbeatResponse response = nm1.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ NodeHeartbeatResponse proto = new NodeHeartbeatResponsePBImpl(
|
|
|
+ ((NodeHeartbeatResponsePBImpl) response).getProto());
|
|
|
+
|
|
|
ByteBuffer tokenBuffer =
|
|
|
- response.getSystemCredentialsForApps().get(app.getApplicationId());
|
|
|
+ YarnServerBuilderUtils
|
|
|
+ .convertFromProtoFormat(proto.getSystemCredentialsForApps())
|
|
|
+ .get(app.getApplicationId());
|
|
|
Assert.assertNotNull(tokenBuffer);
|
|
|
Credentials appCredentials = new Credentials();
|
|
|
DataInputByteBuffer buf = new DataInputByteBuffer();
|
|
@@ -1117,8 +1140,14 @@ public class TestDelegationTokenRenewer {
|
|
|
new MockNM("127.0.0.1:1234", 15120, rm.getResourceTrackerService());
|
|
|
nm1.registerNode();
|
|
|
NodeHeartbeatResponse response = nm1.nodeHeartbeat(true);
|
|
|
+
|
|
|
+ NodeHeartbeatResponse proto = new NodeHeartbeatResponsePBImpl(
|
|
|
+ ((NodeHeartbeatResponsePBImpl) response).getProto());
|
|
|
+
|
|
|
ByteBuffer tokenBuffer =
|
|
|
- response.getSystemCredentialsForApps().get(app.getApplicationId());
|
|
|
+ YarnServerBuilderUtils
|
|
|
+ .convertFromProtoFormat(proto.getSystemCredentialsForApps())
|
|
|
+ .get(app.getApplicationId());
|
|
|
Assert.assertNotNull(tokenBuffer);
|
|
|
Credentials appCredentials = new Credentials();
|
|
|
DataInputByteBuffer buf = new DataInputByteBuffer();
|
|
@@ -1430,7 +1459,7 @@ public class TestDelegationTokenRenewer {
|
|
|
DelegationTokenRenewer dtr = createNewDelegationTokenRenewer(conf, counter);
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
when(mockContext.getSystemCredentialsForApps()).thenReturn(
|
|
|
- new ConcurrentHashMap<ApplicationId, ByteBuffer>());
|
|
|
+ new ConcurrentHashMap<ApplicationId, SystemCredentialsForAppsProto>());
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
InetSocketAddress sockAddr =
|
|
@@ -1444,4 +1473,61 @@ public class TestDelegationTokenRenewer {
|
|
|
delegationTokenRenewer.applicationFinished(
|
|
|
BuilderUtils.newApplicationId(0, 1));
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout = 10000)
|
|
|
+ public void testTokenSequenceNoAfterNewTokenAndRenewal() throws Exception {
|
|
|
+ conf.setBoolean(YarnConfiguration.RM_PROXY_USER_PRIVILEGES_ENABLED, true);
|
|
|
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
+ "kerberos");
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
|
+
|
|
|
+ final Credentials credsx = new Credentials();
|
|
|
+
|
|
|
+ DelegationTokenIdentifier dtId1 = new DelegationTokenIdentifier(
|
|
|
+ new Text("user1"), new Text("renewer"), new Text("user1"));
|
|
|
+ final Token<DelegationTokenIdentifier> expectedToken =
|
|
|
+ new Token<DelegationTokenIdentifier>(dtId1.getBytes(),
|
|
|
+ "password2".getBytes(), dtId1.getKind(), new Text("service2"));
|
|
|
+
|
|
|
+ // fire up the renewer
|
|
|
+ final DelegationTokenRenewer dtr = new DelegationTokenRenewer() {
|
|
|
+ @Override
|
|
|
+ protected Token<?>[] obtainSystemTokensForUser(String user,
|
|
|
+ final Credentials credentials) throws IOException {
|
|
|
+ credentials.addToken(expectedToken.getService(), expectedToken);
|
|
|
+ return new Token<?>[] {expectedToken};
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ RMContext mockContext = mock(RMContext.class);
|
|
|
+ when(mockContext.getSystemCredentialsForApps()).thenReturn(
|
|
|
+ new ConcurrentHashMap<ApplicationId, SystemCredentialsForAppsProto>());
|
|
|
+ ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
+ when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
+ InetSocketAddress sockAddr =
|
|
|
+ InetSocketAddress.createUnresolved("localhost", 1234);
|
|
|
+ when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
|
|
|
+ dtr.setRMContext(mockContext);
|
|
|
+ when(mockContext.getDelegationTokenRenewer()).thenReturn(dtr);
|
|
|
+ dtr.init(conf);
|
|
|
+ dtr.start();
|
|
|
+
|
|
|
+ final ApplicationId appId1 = ApplicationId.newInstance(1234, 1);
|
|
|
+
|
|
|
+ Collection<ApplicationId> appIds = new ArrayList<ApplicationId>(1);
|
|
|
+ appIds.add(appId1);
|
|
|
+
|
|
|
+ dtr.addApplicationSync(appId1, credsx, false, "user1");
|
|
|
+
|
|
|
+ // Ensure incrTokenSequenceNo has been called for new token request
|
|
|
+ Mockito.verify(mockContext, Mockito.times(1)).incrTokenSequenceNo();
|
|
|
+
|
|
|
+ DelegationTokenToRenew dttr = new DelegationTokenToRenew(appIds,
|
|
|
+ expectedToken, conf, 1000, false, "user1");
|
|
|
+
|
|
|
+ dtr.requestNewHdfsDelegationTokenIfNeeded(dttr);
|
|
|
+
|
|
|
+ // Ensure incrTokenSequenceNo has been called for token renewal as well.
|
|
|
+ Mockito.verify(mockContext, Mockito.times(2)).incrTokenSequenceNo();
|
|
|
+ }
|
|
|
}
|