|
@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.client.api.impl;
|
|
|
|
|
|
import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.never;
|
|
|
+import static org.mockito.Mockito.spy;
|
|
|
import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
@@ -39,6 +41,7 @@ import org.apache.commons.io.IOUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.io.DataInputByteBuffer;
|
|
|
import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
+import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
@@ -745,10 +748,13 @@ public class TestYarnClient {
|
|
|
Configuration conf = new YarnConfiguration();
|
|
|
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
|
|
SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
|
|
|
+ TimelineDelegationTokenIdentifier timelineDT =
|
|
|
+ new TimelineDelegationTokenIdentifier();
|
|
|
final Token<TimelineDelegationTokenIdentifier> dToken =
|
|
|
- new Token<TimelineDelegationTokenIdentifier>();
|
|
|
+ new Token<TimelineDelegationTokenIdentifier>(
|
|
|
+ timelineDT.getBytes(), new byte[0], timelineDT.getKind(), new Text());
|
|
|
// crate a mock client
|
|
|
- YarnClientImpl client = new YarnClientImpl() {
|
|
|
+ YarnClientImpl client = spy(new YarnClientImpl() {
|
|
|
@Override
|
|
|
protected void serviceInit(Configuration conf) throws Exception {
|
|
|
if (getConfig().getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
|
|
@@ -784,34 +790,48 @@ public class TestYarnClient {
|
|
|
public boolean isSecurityEnabled() {
|
|
|
return true;
|
|
|
}
|
|
|
- };
|
|
|
+ });
|
|
|
client.init(conf);
|
|
|
client.start();
|
|
|
- ApplicationSubmissionContext context =
|
|
|
- mock(ApplicationSubmissionContext.class);
|
|
|
- ApplicationId applicationId = ApplicationId.newInstance(0, 1);
|
|
|
- when(context.getApplicationId()).thenReturn(applicationId);
|
|
|
- DataOutputBuffer dob = new DataOutputBuffer();
|
|
|
- Credentials credentials = new Credentials();
|
|
|
- credentials.writeTokenStorageToStream(dob);
|
|
|
- ByteBuffer tokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
|
|
|
- ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
|
|
|
- null, null, null, null, tokens, null);
|
|
|
- when(context.getAMContainerSpec()).thenReturn(clc);
|
|
|
- client.submitApplication(context);
|
|
|
- // Check whether token is added or not
|
|
|
- credentials = new Credentials();
|
|
|
- DataInputByteBuffer dibb = new DataInputByteBuffer();
|
|
|
- tokens = clc.getTokens();
|
|
|
- if (tokens != null) {
|
|
|
- dibb.reset(tokens);
|
|
|
- credentials.readTokenStorageStream(dibb);
|
|
|
- tokens.rewind();
|
|
|
+ try {
|
|
|
+ // when i == 0, timeline DT already exists, no need to get one more
|
|
|
+ // when i == 1, timeline DT doesn't exist, need to get one more
|
|
|
+ for (int i = 0; i < 2; ++i) {
|
|
|
+ ApplicationSubmissionContext context =
|
|
|
+ mock(ApplicationSubmissionContext.class);
|
|
|
+ ApplicationId applicationId = ApplicationId.newInstance(0, i + 1);
|
|
|
+ when(context.getApplicationId()).thenReturn(applicationId);
|
|
|
+ DataOutputBuffer dob = new DataOutputBuffer();
|
|
|
+ Credentials credentials = new Credentials();
|
|
|
+ if (i == 0) {
|
|
|
+ credentials.addToken(client.timelineService, dToken);
|
|
|
+ }
|
|
|
+ credentials.writeTokenStorageToStream(dob);
|
|
|
+ ByteBuffer tokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
|
|
|
+ ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
|
|
|
+ null, null, null, null, tokens, null);
|
|
|
+ when(context.getAMContainerSpec()).thenReturn(clc);
|
|
|
+ client.submitApplication(context);
|
|
|
+ if (i == 0) {
|
|
|
+ // GetTimelineDelegationToken shouldn't be called
|
|
|
+ verify(client, never()).getTimelineDelegationToken();
|
|
|
+ }
|
|
|
+ // In either way, token should be there
|
|
|
+ credentials = new Credentials();
|
|
|
+ DataInputByteBuffer dibb = new DataInputByteBuffer();
|
|
|
+ tokens = clc.getTokens();
|
|
|
+ if (tokens != null) {
|
|
|
+ dibb.reset(tokens);
|
|
|
+ credentials.readTokenStorageStream(dibb);
|
|
|
+ tokens.rewind();
|
|
|
+ }
|
|
|
+ Collection<Token<? extends TokenIdentifier>> dTokens =
|
|
|
+ credentials.getAllTokens();
|
|
|
+ Assert.assertEquals(1, dTokens.size());
|
|
|
+ Assert.assertEquals(dToken, dTokens.iterator().next());
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ client.stop();
|
|
|
}
|
|
|
- Collection<Token<? extends TokenIdentifier>> dTokens =
|
|
|
- credentials.getAllTokens();
|
|
|
- Assert.assertEquals(1, dTokens.size());
|
|
|
- Assert.assertEquals(dToken, dTokens.iterator().next());
|
|
|
- client.stop();
|
|
|
}
|
|
|
}
|