1
0

TestDelegationToken.java 5.8 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154
  1. /**
  2. * Licensed to the Apache Software Foundation (ASF) under one
  3. * or more contributor license agreements. See the NOTICE file
  4. * distributed with this work for additional information
  5. * regarding copyright ownership. The ASF licenses this file
  6. * to you under the Apache License, Version 2.0 (the
  7. * "License"); you may not use this file except in compliance
  8. * with the License. You may obtain a copy of the License at
  9. *
  10. * http://www.apache.org/licenses/LICENSE-2.0
  11. *
  12. * Unless required by applicable law or agreed to in writing, software
  13. * distributed under the License is distributed on an "AS IS" BASIS,
  14. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  15. * See the License for the specific language governing permissions and
  16. * limitations under the License.
  17. */
  18. package org.apache.hadoop.hdfs.security;
  19. import java.io.ByteArrayInputStream;
  20. import java.io.DataInputStream;
  21. import java.io.IOException;
  22. import java.security.PrivilegedExceptionAction;
  23. import junit.framework.Assert;
  24. import org.apache.hadoop.conf.Configuration;
  25. import org.apache.hadoop.fs.FileSystem;
  26. import org.apache.hadoop.hdfs.DFSConfigKeys;
  27. import org.apache.hadoop.hdfs.DistributedFileSystem;
  28. import org.apache.hadoop.hdfs.MiniDFSCluster;
  29. import org.apache.hadoop.io.Text;
  30. import org.apache.hadoop.security.AccessControlException;
  31. import org.apache.hadoop.security.UserGroupInformation;
  32. import org.apache.hadoop.security.token.Token;
  33. import org.apache.hadoop.security.token.SecretManager.InvalidToken;
  34. import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
  35. import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
  36. import org.junit.After;
  37. import org.junit.Before;
  38. import org.junit.Test;
  39. import org.mortbay.log.Log;
  40. public class TestDelegationToken {
  41. private MiniDFSCluster cluster;
  42. Configuration config;
  43. @Before
  44. public void setUp() throws Exception {
  45. config = new Configuration();
  46. config.setLong(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000);
  47. config.setLong(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 5000);
  48. FileSystem.setDefaultUri(config, "hdfs://localhost:" + "0");
  49. cluster = new MiniDFSCluster(0, config, 1, true, true, true, null, null, null, null);
  50. cluster.waitActive();
  51. }
  52. @After
  53. public void tearDown() throws Exception {
  54. if(cluster!=null) {
  55. cluster.shutdown();
  56. }
  57. }
  58. private Token<DelegationTokenIdentifier> generateDelegationToken(
  59. String owner, String renewer) {
  60. DelegationTokenSecretManager dtSecretManager = cluster.getNameNode()
  61. .getNamesystem().getDelegationTokenSecretManager();
  62. DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(new Text(
  63. owner), new Text(renewer), null);
  64. return new Token<DelegationTokenIdentifier>(dtId, dtSecretManager);
  65. }
  66. @Test
  67. public void testDelegationTokenSecretManager() throws Exception {
  68. DelegationTokenSecretManager dtSecretManager = cluster.getNameNode()
  69. .getNamesystem().getDelegationTokenSecretManager();
  70. Token<DelegationTokenIdentifier> token = generateDelegationToken(
  71. "SomeUser", "JobTracker");
  72. // Fake renewer should not be able to renew
  73. try {
  74. dtSecretManager.renewToken(token, "FakeRenewer");
  75. Assert.fail("should have failed");
  76. } catch (AccessControlException ace) {
  77. // PASS
  78. }
  79. dtSecretManager.renewToken(token, "JobTracker");
  80. DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
  81. byte[] tokenId = token.getIdentifier();
  82. identifier.readFields(new DataInputStream(
  83. new ByteArrayInputStream(tokenId)));
  84. Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
  85. Log.info("Sleep to expire the token");
  86. Thread.sleep(6000);
  87. //Token should be expired
  88. try {
  89. dtSecretManager.retrievePassword(identifier);
  90. //Should not come here
  91. Assert.fail("Token should have expired");
  92. } catch (InvalidToken e) {
  93. //Success
  94. }
  95. dtSecretManager.renewToken(token, "JobTracker");
  96. Log.info("Sleep beyond the max lifetime");
  97. Thread.sleep(5000);
  98. try {
  99. dtSecretManager.renewToken(token, "JobTracker");
  100. Assert.fail("should have been expired");
  101. } catch (InvalidToken it) {
  102. // PASS
  103. }
  104. }
  105. @Test
  106. public void testCancelDelegationToken() throws Exception {
  107. DelegationTokenSecretManager dtSecretManager = cluster.getNameNode()
  108. .getNamesystem().getDelegationTokenSecretManager();
  109. Token<DelegationTokenIdentifier> token = generateDelegationToken(
  110. "SomeUser", "JobTracker");
  111. //Fake renewer should not be able to renew
  112. try {
  113. dtSecretManager.cancelToken(token, "FakeCanceller");
  114. Assert.fail("should have failed");
  115. } catch (AccessControlException ace) {
  116. // PASS
  117. }
  118. dtSecretManager.cancelToken(token, "JobTracker");
  119. try {
  120. dtSecretManager.renewToken(token, "JobTracker");
  121. Assert.fail("should have failed");
  122. } catch (InvalidToken it) {
  123. // PASS
  124. }
  125. }
  126. @Test
  127. public void testDelegationTokenDFSApi() throws Exception {
  128. DelegationTokenSecretManager dtSecretManager = cluster.getNameNode()
  129. .getNamesystem().getDelegationTokenSecretManager();
  130. DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
  131. Token<DelegationTokenIdentifier> token = dfs.getDelegationToken(new Text("JobTracker"));
  132. DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
  133. byte[] tokenId = token.getIdentifier();
  134. identifier.readFields(new DataInputStream(
  135. new ByteArrayInputStream(tokenId)));
  136. Log.info("A valid token should have non-null password, and should be renewed successfully");
  137. Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
  138. dtSecretManager.renewToken(token, "JobTracker");
  139. }
  140. }