|
@@ -31,13 +31,24 @@ import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
+import java.nio.ByteBuffer;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.concurrent.BlockingQueue;
|
|
|
import java.util.concurrent.BrokenBarrierException;
|
|
|
import java.util.concurrent.CyclicBarrier;
|
|
|
+import java.util.concurrent.LinkedBlockingQueue;
|
|
|
+import java.util.concurrent.ThreadPoolExecutor;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+
|
|
|
+import junit.framework.Assert;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
@@ -46,16 +57,29 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenRenewer;
|
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
-import org.apache.hadoop.service.Service.STATE;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
+import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
|
+import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
+import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
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.server.resourcemanager.ClientRMService;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
@@ -66,14 +90,18 @@ import org.mockito.stubbing.Answer;
|
|
|
|
|
|
/**
|
|
|
* unit test -
|
|
|
- * tests addition/deletion/cancelation of renewals of delegation tokens
|
|
|
+ * tests addition/deletion/cancellation of renewals of delegation tokens
|
|
|
*
|
|
|
*/
|
|
|
+@SuppressWarnings("rawtypes")
|
|
|
public class TestDelegationTokenRenewer {
|
|
|
private static final Log LOG =
|
|
|
LogFactory.getLog(TestDelegationTokenRenewer.class);
|
|
|
private static final Text KIND = new Text("TestDelegationTokenRenewer.Token");
|
|
|
|
|
|
+ private static BlockingQueue<Event> eventQueue;
|
|
|
+ private static volatile AtomicInteger counter;
|
|
|
+ private static AsyncDispatcher dispatcher;
|
|
|
public static class Renewer extends TokenRenewer {
|
|
|
private static int counter = 0;
|
|
|
private static Token<?> lastRenewed = null;
|
|
@@ -143,11 +171,20 @@ public class TestDelegationTokenRenewer {
|
|
|
|
|
|
@Before
|
|
|
public void setUp() throws Exception {
|
|
|
+ counter = new AtomicInteger(0);
|
|
|
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
+ "kerberos");
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
|
+ eventQueue = new LinkedBlockingQueue<Event>();
|
|
|
+ dispatcher = new AsyncDispatcher(eventQueue);
|
|
|
Renewer.reset();
|
|
|
- delegationTokenRenewer = new DelegationTokenRenewer();
|
|
|
+ delegationTokenRenewer = createNewDelegationTokenRenewer(conf, counter);
|
|
|
delegationTokenRenewer.init(conf);
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
+ when(mockContext.getDelegationTokenRenewer()).thenReturn(
|
|
|
+ delegationTokenRenewer);
|
|
|
+ when(mockContext.getDispatcher()).thenReturn(dispatcher);
|
|
|
when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
InetSocketAddress sockAddr =
|
|
|
InetSocketAddress.createUnresolved("localhost", 1234);
|
|
@@ -285,7 +322,7 @@ public class TestDelegationTokenRenewer {
|
|
|
* @throws IOException
|
|
|
* @throws URISyntaxException
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testDTRenewal () throws Exception {
|
|
|
MyFS dfs = (MyFS)FileSystem.get(conf);
|
|
|
LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
|
|
@@ -316,8 +353,9 @@ public class TestDelegationTokenRenewer {
|
|
|
// register the tokens for renewal
|
|
|
ApplicationId applicationId_0 =
|
|
|
BuilderUtils.newApplicationId(0, 0);
|
|
|
- delegationTokenRenewer.addApplication(applicationId_0, ts, true);
|
|
|
-
|
|
|
+ delegationTokenRenewer.addApplication(applicationId_0, ts, true, false);
|
|
|
+ waitForEventsToGetProcessed(delegationTokenRenewer);
|
|
|
+
|
|
|
// first 3 initial renewals + 1 real
|
|
|
int numberOfExpectedRenewals = 3+1;
|
|
|
|
|
@@ -355,9 +393,10 @@ public class TestDelegationTokenRenewer {
|
|
|
|
|
|
|
|
|
ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1);
|
|
|
- delegationTokenRenewer.addApplication(applicationId_1, ts, true);
|
|
|
+ delegationTokenRenewer.addApplication(applicationId_1, ts, true, false);
|
|
|
+ waitForEventsToGetProcessed(delegationTokenRenewer);
|
|
|
delegationTokenRenewer.applicationFinished(applicationId_1);
|
|
|
-
|
|
|
+ waitForEventsToGetProcessed(delegationTokenRenewer);
|
|
|
numberOfExpectedRenewals = Renewer.counter; // number of renewals so far
|
|
|
try {
|
|
|
Thread.sleep(6*1000); // sleep 6 seconds, so it has time to renew
|
|
@@ -377,8 +416,8 @@ public class TestDelegationTokenRenewer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testInvalidDTWithAddApplication() throws Exception {
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testAppRejectionWithCancelledDelegationToken() throws Exception {
|
|
|
MyFS dfs = (MyFS)FileSystem.get(conf);
|
|
|
LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
|
|
|
|
|
@@ -390,12 +429,21 @@ public class TestDelegationTokenRenewer {
|
|
|
|
|
|
// register the tokens for renewal
|
|
|
ApplicationId appId = BuilderUtils.newApplicationId(0, 0);
|
|
|
- try {
|
|
|
- delegationTokenRenewer.addApplication(appId, ts, true);
|
|
|
- fail("App submission with a cancelled token should have failed");
|
|
|
- } catch (InvalidToken e) {
|
|
|
- // expected
|
|
|
+ delegationTokenRenewer.addApplication(appId, ts, true, false);
|
|
|
+ int waitCnt = 20;
|
|
|
+ while (waitCnt-- >0) {
|
|
|
+ if (!eventQueue.isEmpty()) {
|
|
|
+ Event evt = eventQueue.take();
|
|
|
+ if (evt.getType() == RMAppEventType.APP_REJECTED) {
|
|
|
+ Assert.assertTrue(
|
|
|
+ ((RMAppEvent) evt).getApplicationId().equals(appId));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ Thread.sleep(500);
|
|
|
+ }
|
|
|
}
|
|
|
+ fail("App submission with a cancelled token should have failed");
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -408,7 +456,7 @@ public class TestDelegationTokenRenewer {
|
|
|
* @throws IOException
|
|
|
* @throws URISyntaxException
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testDTRenewalWithNoCancel () throws Exception {
|
|
|
MyFS dfs = (MyFS)FileSystem.get(conf);
|
|
|
LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
|
|
@@ -425,9 +473,10 @@ public class TestDelegationTokenRenewer {
|
|
|
|
|
|
|
|
|
ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1);
|
|
|
- delegationTokenRenewer.addApplication(applicationId_1, ts, false);
|
|
|
+ delegationTokenRenewer.addApplication(applicationId_1, ts, false, false);
|
|
|
+ waitForEventsToGetProcessed(delegationTokenRenewer);
|
|
|
delegationTokenRenewer.applicationFinished(applicationId_1);
|
|
|
-
|
|
|
+ waitForEventsToGetProcessed(delegationTokenRenewer);
|
|
|
int numberOfExpectedRenewals = Renewer.counter; // number of renewals so far
|
|
|
try {
|
|
|
Thread.sleep(6*1000); // sleep 6 seconds, so it has time to renew
|
|
@@ -454,9 +503,8 @@ public class TestDelegationTokenRenewer {
|
|
|
* @throws IOException
|
|
|
* @throws URISyntaxException
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testDTKeepAlive1 () throws Exception {
|
|
|
- DelegationTokenRenewer localDtr = new DelegationTokenRenewer();
|
|
|
Configuration lconf = new Configuration(conf);
|
|
|
lconf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
|
|
|
//Keep tokens alive for 6 seconds.
|
|
@@ -465,10 +513,15 @@ public class TestDelegationTokenRenewer {
|
|
|
lconf.setLong(
|
|
|
YarnConfiguration.RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS,
|
|
|
1000l);
|
|
|
+ DelegationTokenRenewer localDtr =
|
|
|
+ createNewDelegationTokenRenewer(lconf, counter);
|
|
|
localDtr.init(lconf);
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
+ when(mockContext.getDelegationTokenRenewer()).thenReturn(
|
|
|
+ localDtr);
|
|
|
+ when(mockContext.getDispatcher()).thenReturn(dispatcher);
|
|
|
InetSocketAddress sockAddr =
|
|
|
InetSocketAddress.createUnresolved("localhost", 1234);
|
|
|
when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
|
|
@@ -487,16 +540,25 @@ public class TestDelegationTokenRenewer {
|
|
|
|
|
|
// register the tokens for renewal
|
|
|
ApplicationId applicationId_0 = BuilderUtils.newApplicationId(0, 0);
|
|
|
- localDtr.addApplication(applicationId_0, ts, true);
|
|
|
+ localDtr.addApplication(applicationId_0, ts, true, false);
|
|
|
+ waitForEventsToGetProcessed(localDtr);
|
|
|
+ if (!eventQueue.isEmpty()){
|
|
|
+ Event evt = eventQueue.take();
|
|
|
+ if (evt instanceof RMAppEvent) {
|
|
|
+ Assert.assertEquals(((RMAppEvent)evt).getType(), RMAppEventType.START);
|
|
|
+ } else {
|
|
|
+ fail("RMAppEvent.START was expected!!");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
localDtr.applicationFinished(applicationId_0);
|
|
|
-
|
|
|
- Thread.sleep(3000l);
|
|
|
+ waitForEventsToGetProcessed(localDtr);
|
|
|
|
|
|
//Token should still be around. Renewal should not fail.
|
|
|
token1.renew(lconf);
|
|
|
|
|
|
//Allow the keepalive time to run out
|
|
|
- Thread.sleep(6000l);
|
|
|
+ Thread.sleep(10000l);
|
|
|
|
|
|
//The token should have been cancelled at this point. Renewal will fail.
|
|
|
try {
|
|
@@ -518,9 +580,8 @@ public class TestDelegationTokenRenewer {
|
|
|
* @throws IOException
|
|
|
* @throws URISyntaxException
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=60000)
|
|
|
public void testDTKeepAlive2() throws Exception {
|
|
|
- DelegationTokenRenewer localDtr = new DelegationTokenRenewer();
|
|
|
Configuration lconf = new Configuration(conf);
|
|
|
lconf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
|
|
|
//Keep tokens alive for 6 seconds.
|
|
@@ -529,10 +590,15 @@ public class TestDelegationTokenRenewer {
|
|
|
lconf.setLong(
|
|
|
YarnConfiguration.RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS,
|
|
|
1000l);
|
|
|
+ DelegationTokenRenewer localDtr =
|
|
|
+ createNewDelegationTokenRenewer(conf, counter);
|
|
|
localDtr.init(lconf);
|
|
|
RMContext mockContext = mock(RMContext.class);
|
|
|
ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
+ when(mockContext.getDelegationTokenRenewer()).thenReturn(
|
|
|
+ localDtr);
|
|
|
+ when(mockContext.getDispatcher()).thenReturn(dispatcher);
|
|
|
InetSocketAddress sockAddr =
|
|
|
InetSocketAddress.createUnresolved("localhost", 1234);
|
|
|
when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
|
|
@@ -551,22 +617,18 @@ public class TestDelegationTokenRenewer {
|
|
|
|
|
|
// register the tokens for renewal
|
|
|
ApplicationId applicationId_0 = BuilderUtils.newApplicationId(0, 0);
|
|
|
- localDtr.addApplication(applicationId_0, ts, true);
|
|
|
+ localDtr.addApplication(applicationId_0, ts, true, false);
|
|
|
localDtr.applicationFinished(applicationId_0);
|
|
|
-
|
|
|
- Thread.sleep(4000l);
|
|
|
-
|
|
|
+ waitForEventsToGetProcessed(delegationTokenRenewer);
|
|
|
//Send another keep alive.
|
|
|
localDtr.updateKeepAliveApplications(Collections
|
|
|
.singletonList(applicationId_0));
|
|
|
//Renewal should not fail.
|
|
|
token1.renew(lconf);
|
|
|
-
|
|
|
//Token should be around after this.
|
|
|
Thread.sleep(4500l);
|
|
|
//Renewal should not fail. - ~1.5 seconds for keepalive timeout.
|
|
|
token1.renew(lconf);
|
|
|
-
|
|
|
//Allow the keepalive time to run out
|
|
|
Thread.sleep(3000l);
|
|
|
//The token should have been cancelled at this point. Renewal will fail.
|
|
@@ -575,61 +637,127 @@ public class TestDelegationTokenRenewer {
|
|
|
fail("Renewal of cancelled token should have failed");
|
|
|
} catch (InvalidToken ite) {}
|
|
|
}
|
|
|
-
|
|
|
- @Test(timeout=20000)
|
|
|
- public void testConncurrentAddApplication()
|
|
|
- throws IOException, InterruptedException, BrokenBarrierException {
|
|
|
- final CyclicBarrier startBarrier = new CyclicBarrier(2);
|
|
|
- final CyclicBarrier endBarrier = new CyclicBarrier(2);
|
|
|
-
|
|
|
- // this token uses barriers to block during renew
|
|
|
- final Credentials creds1 = new Credentials();
|
|
|
- final Token<?> token1 = mock(Token.class);
|
|
|
- creds1.addToken(new Text("token"), token1);
|
|
|
- doReturn(true).when(token1).isManaged();
|
|
|
- doAnswer(new Answer<Long>() {
|
|
|
- public Long answer(InvocationOnMock invocation)
|
|
|
- throws InterruptedException, BrokenBarrierException {
|
|
|
- startBarrier.await();
|
|
|
- endBarrier.await();
|
|
|
- return Long.MAX_VALUE;
|
|
|
- }}).when(token1).renew(any(Configuration.class));
|
|
|
-
|
|
|
- // this dummy token fakes renewing
|
|
|
- final Credentials creds2 = new Credentials();
|
|
|
- final Token<?> token2 = mock(Token.class);
|
|
|
- creds2.addToken(new Text("token"), token2);
|
|
|
- doReturn(true).when(token2).isManaged();
|
|
|
- doReturn(Long.MAX_VALUE).when(token2).renew(any(Configuration.class));
|
|
|
-
|
|
|
- // fire up the renewer
|
|
|
- final DelegationTokenRenewer dtr = new DelegationTokenRenewer();
|
|
|
- dtr.init(conf);
|
|
|
- RMContext mockContext = mock(RMContext.class);
|
|
|
- ClientRMService mockClientRMService = mock(ClientRMService.class);
|
|
|
- when(mockContext.getClientRMService()).thenReturn(mockClientRMService);
|
|
|
- InetSocketAddress sockAddr =
|
|
|
- InetSocketAddress.createUnresolved("localhost", 1234);
|
|
|
- when(mockClientRMService.getBindAddress()).thenReturn(sockAddr);
|
|
|
- dtr.setRMContext(mockContext);
|
|
|
- dtr.start();
|
|
|
-
|
|
|
- // submit a job that blocks during renewal
|
|
|
- Thread submitThread = new Thread() {
|
|
|
+
|
|
|
+ private DelegationTokenRenewer createNewDelegationTokenRenewer(
|
|
|
+ Configuration conf, final AtomicInteger counter) {
|
|
|
+ return new DelegationTokenRenewer() {
|
|
|
+
|
|
|
@Override
|
|
|
- public void run() {
|
|
|
- try {
|
|
|
- dtr.addApplication(mock(ApplicationId.class), creds1, false);
|
|
|
- } catch (IOException e) {}
|
|
|
+ protected ThreadPoolExecutor
|
|
|
+ createNewThreadPoolService(Configuration conf) {
|
|
|
+ ThreadPoolExecutor pool =
|
|
|
+ new ThreadPoolExecutor(5, 5, 3L, TimeUnit.SECONDS,
|
|
|
+ new LinkedBlockingQueue<Runnable>()) {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void afterExecute(Runnable r, Throwable t) {
|
|
|
+ counter.decrementAndGet();
|
|
|
+ super.afterExecute(r, t);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void execute(Runnable command) {
|
|
|
+ counter.incrementAndGet();
|
|
|
+ super.execute(command);
|
|
|
+ }
|
|
|
+ };
|
|
|
+ return pool;
|
|
|
}
|
|
|
};
|
|
|
- submitThread.start();
|
|
|
-
|
|
|
+ }
|
|
|
+
|
|
|
+ private void waitForEventsToGetProcessed(DelegationTokenRenewer dtr)
|
|
|
+ throws InterruptedException {
|
|
|
+ int wait = 40;
|
|
|
+ while (wait-- > 0
|
|
|
+ && counter.get() > 0) {
|
|
|
+ Thread.sleep(200);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout=20000)
|
|
|
+ public void testConcurrentAddApplication()
|
|
|
+ throws IOException, InterruptedException, BrokenBarrierException {
|
|
|
+ final CyclicBarrier startBarrier = new CyclicBarrier(2);
|
|
|
+ final CyclicBarrier endBarrier = new CyclicBarrier(2);
|
|
|
+
|
|
|
+ // this token uses barriers to block during renew
|
|
|
+ final Credentials creds1 = new Credentials();
|
|
|
+ final Token<?> token1 = mock(Token.class);
|
|
|
+ creds1.addToken(new Text("token"), token1);
|
|
|
+ doReturn(true).when(token1).isManaged();
|
|
|
+ doAnswer(new Answer<Long>() {
|
|
|
+ public Long answer(InvocationOnMock invocation)
|
|
|
+ throws InterruptedException, BrokenBarrierException {
|
|
|
+ startBarrier.await();
|
|
|
+ endBarrier.await();
|
|
|
+ return Long.MAX_VALUE;
|
|
|
+ }}).when(token1).renew(any(Configuration.class));
|
|
|
+
|
|
|
+ // this dummy token fakes renewing
|
|
|
+ final Credentials creds2 = new Credentials();
|
|
|
+ final Token<?> token2 = mock(Token.class);
|
|
|
+ creds2.addToken(new Text("token"), token2);
|
|
|
+ doReturn(true).when(token2).isManaged();
|
|
|
+ doReturn(Long.MAX_VALUE).when(token2).renew(any(Configuration.class));
|
|
|
+
|
|
|
+ // fire up the renewer
|
|
|
+ final DelegationTokenRenewer dtr =
|
|
|
+ createNewDelegationTokenRenewer(conf, counter);
|
|
|
+ dtr.init(conf);
|
|
|
+ RMContext mockContext = mock(RMContext.class);
|
|
|
+ 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.start();
|
|
|
+ // submit a job that blocks during renewal
|
|
|
+ Thread submitThread = new Thread() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ dtr.addApplication(mock(ApplicationId.class), creds1, false, false);
|
|
|
+ }
|
|
|
+ };
|
|
|
+ submitThread.start();
|
|
|
+
|
|
|
// wait till 1st submit blocks, then submit another
|
|
|
- startBarrier.await();
|
|
|
- dtr.addApplication(mock(ApplicationId.class), creds2, false);
|
|
|
- // signal 1st to complete
|
|
|
- endBarrier.await();
|
|
|
- submitThread.join();
|
|
|
+ startBarrier.await();
|
|
|
+ dtr.addApplication(mock(ApplicationId.class), creds2, false, false);
|
|
|
+ // signal 1st to complete
|
|
|
+ endBarrier.await();
|
|
|
+ submitThread.join();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout=20000)
|
|
|
+ public void testAppSubmissionWithInvalidDelegationToken() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.set(
|
|
|
+ CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
+ "kerberos");
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
|
+ MockRM rm = new MockRM(conf);
|
|
|
+ ByteBuffer tokens = ByteBuffer.wrap("BOGUS".getBytes());
|
|
|
+ ContainerLaunchContext amContainer =
|
|
|
+ ContainerLaunchContext.newInstance(
|
|
|
+ new HashMap<String, LocalResource>(), new HashMap<String, String>(),
|
|
|
+ new ArrayList<String>(), new HashMap<String, ByteBuffer>(), tokens,
|
|
|
+ new HashMap<ApplicationAccessType, String>());
|
|
|
+ ApplicationSubmissionContext appSubContext =
|
|
|
+ ApplicationSubmissionContext.newInstance(
|
|
|
+ ApplicationId.newInstance(1234121, 0),
|
|
|
+ "BOGUS", "default", Priority.UNDEFINED, amContainer, false,
|
|
|
+ true, 1, Resource.newInstance(1024, 1), "BOGUS");
|
|
|
+ SubmitApplicationRequest request =
|
|
|
+ SubmitApplicationRequest.newInstance(appSubContext);
|
|
|
+ try {
|
|
|
+ rm.getClientRMService().submitApplication(request);
|
|
|
+ fail("Error was excepted.");
|
|
|
+ } catch (YarnException e) {
|
|
|
+ Assert.assertTrue(e.getMessage().contains(
|
|
|
+ "Bad header found in token storage"));
|
|
|
+ }
|
|
|
}
|
|
|
}
|