@Test(timeout = 60000)
  public void testAppRejectionWithCancelledDelegationToken() throws Exception {
    MyFS dfs = (MyFS) FileSystem.get(conf);
    LOG.info("dfs=" + (Object) dfs.hashCode() + ";conf=" + conf.hashCode());

    MyToken token = dfs.getDelegationToken(new Text("user1"));
    token.cancelToken();

    Credentials ts = new Credentials();
    ts.addToken(token.getKind(), token);

    // register the tokens for renewal
    ApplicationId appId = BuilderUtils.newApplicationId(0, 0);
    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");
  }
  /**
   * Basic idea of the test: 1. register a token for 2 seconds with no cancel at the end 2. cancel
   * it immediately 3. Sleep and check that the 2 seconds renew didn't happen (totally 5 renewals)
   * 4. check cancellation
   *
   * @throws IOException
   * @throws URISyntaxException
   */
  @Test(timeout = 60000)
  public void testDTRenewalWithNoCancel() throws Exception {
    MyFS dfs = (MyFS) FileSystem.get(conf);
    LOG.info("dfs=" + (Object) dfs.hashCode() + ";conf=" + conf.hashCode());

    Credentials ts = new Credentials();
    MyToken token1 = dfs.getDelegationToken(new Text("user1"));

    // to cause this one to be set for renew in 2 secs
    Renewer.tokenToRenewIn2Sec = token1;
    LOG.info("token=" + token1 + " should be renewed for 2 secs");

    String nn1 = DelegationTokenRenewer.SCHEME + "://host1:0";
    ts.addToken(new Text(nn1), token1);

    ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1);
    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
    } catch (InterruptedException e) {
    }
    LOG.info("Counter = " + Renewer.counter + ";t=" + Renewer.lastRenewed);

    // counter and the token should still be the old ones
    assertEquals(
        "renew wasn't called as many times as expected", numberOfExpectedRenewals, Renewer.counter);

    // also renewing of the canceled token should not fail, because it has not
    // been canceled
    token1.renew(conf);
  }
  /**
   * Basic idea of the test: 0. Setup token KEEP_ALIVE 1. create tokens. 2. register them for
   * renewal - to be cancelled on app complete 3. Complete app. 4. Verify token is alive within the
   * KEEP_ALIVE time 5. Verify token has been cancelled after the KEEP_ALIVE_TIME
   *
   * @throws IOException
   * @throws URISyntaxException
   */
  @Test(timeout = 60000)
  public void testDTKeepAlive1() throws Exception {
    Configuration lconf = new Configuration(conf);
    lconf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
    // Keep tokens alive for 6 seconds.
    lconf.setLong(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 6000l);
    // Try removing tokens every second.
    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);
    localDtr.setRMContext(mockContext);
    localDtr.start();

    MyFS dfs = (MyFS) FileSystem.get(lconf);
    LOG.info("dfs=" + (Object) dfs.hashCode() + ";conf=" + lconf.hashCode());

    Credentials ts = new Credentials();
    // get the delegation tokens
    MyToken token1 = dfs.getDelegationToken(new Text("user1"));

    String nn1 = DelegationTokenRenewer.SCHEME + "://host1:0";
    ts.addToken(new Text(nn1), token1);

    // register the tokens for renewal
    ApplicationId applicationId_0 = BuilderUtils.newApplicationId(0, 0);
    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);
    waitForEventsToGetProcessed(localDtr);

    // Token should still be around. Renewal should not fail.
    token1.renew(lconf);

    // Allow the keepalive time to run out
    Thread.sleep(10000l);

    // The token should have been cancelled at this point. Renewal will fail.
    try {
      token1.renew(lconf);
      fail("Renewal of cancelled token should have failed");
    } catch (InvalidToken ite) {
    }
  }
  /**
   * Basic idea of the test: 1. create tokens. 2. Mark one of them to be renewed in 2 seconds
   * (instead of 24 hours) 3. register them for renewal 4. sleep for 3 seconds 5. count number of
   * renewals (should 3 initial ones + one extra) 6. register another token for 2 seconds 7. cancel
   * it immediately 8. Sleep and check that the 2 seconds renew didn't happen (totally 5 renewals)
   * 9. check cancellation
   *
   * @throws IOException
   * @throws URISyntaxException
   */
  @Test(timeout = 60000)
  public void testDTRenewal() throws Exception {
    MyFS dfs = (MyFS) FileSystem.get(conf);
    LOG.info("dfs=" + (Object) dfs.hashCode() + ";conf=" + conf.hashCode());
    // Test 1. - add three tokens - make sure exactly one get's renewed

    // get the delegation tokens
    MyToken token1, token2, token3;
    token1 = dfs.getDelegationToken(new Text("user1"));
    token2 = dfs.getDelegationToken(new Text("user2"));
    token3 = dfs.getDelegationToken(new Text("user3"));

    // to cause this one to be set for renew in 2 secs
    Renewer.tokenToRenewIn2Sec = token1;
    LOG.info("token=" + token1 + " should be renewed for 2 secs");

    // three distinct Namenodes
    String nn1 = DelegationTokenRenewer.SCHEME + "://host1:0";
    String nn2 = DelegationTokenRenewer.SCHEME + "://host2:0";
    String nn3 = DelegationTokenRenewer.SCHEME + "://host3:0";

    Credentials ts = new Credentials();

    // register the token for renewal
    ts.addToken(new Text(nn1), token1);
    ts.addToken(new Text(nn2), token2);
    ts.addToken(new Text(nn3), token3);

    // register the tokens for renewal
    ApplicationId applicationId_0 = BuilderUtils.newApplicationId(0, 0);
    delegationTokenRenewer.addApplication(applicationId_0, ts, true, false);
    waitForEventsToGetProcessed(delegationTokenRenewer);

    // first 3 initial renewals + 1 real
    int numberOfExpectedRenewals = 3 + 1;

    int attempts = 10;
    while (attempts-- > 0) {
      try {
        Thread.sleep(3 * 1000); // sleep 3 seconds, so it has time to renew
      } catch (InterruptedException e) {
      }

      // since we cannot guarantee timely execution - let's give few chances
      if (Renewer.counter == numberOfExpectedRenewals) break;
    }

    LOG.info(
        "dfs=" + dfs.hashCode() + ";Counter = " + Renewer.counter + ";t=" + Renewer.lastRenewed);
    assertEquals(
        "renew wasn't called as many times as expected(4):",
        numberOfExpectedRenewals,
        Renewer.counter);
    assertEquals("most recently renewed token mismatch", Renewer.lastRenewed, token1);

    // Test 2.
    // add another token ( that expires in 2 secs). Then remove it, before
    // time is up.
    // Wait for 3 secs , and make sure no renews were called
    ts = new Credentials();
    MyToken token4 = dfs.getDelegationToken(new Text("user4"));

    // to cause this one to be set for renew in 2 secs
    Renewer.tokenToRenewIn2Sec = token4;
    LOG.info("token=" + token4 + " should be renewed for 2 secs");

    String nn4 = DelegationTokenRenewer.SCHEME + "://host4:0";
    ts.addToken(new Text(nn4), token4);

    ApplicationId applicationId_1 = BuilderUtils.newApplicationId(0, 1);
    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
    } catch (InterruptedException e) {
    }
    LOG.info("Counter = " + Renewer.counter + ";t=" + Renewer.lastRenewed);

    // counter and the token should stil be the old ones
    assertEquals(
        "renew wasn't called as many times as expected", numberOfExpectedRenewals, Renewer.counter);

    // also renewing of the cancelled token should fail
    try {
      token4.renew(conf);
      fail("Renewal of cancelled token should have failed");
    } catch (InvalidToken ite) {
      // expected
    }
  }