|
@@ -17,6 +17,8 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
|
|
+
|
|
|
import junit.framework.TestCase;
|
|
|
import java.io.*;
|
|
|
import java.net.URI;
|
|
@@ -27,6 +29,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
|
@@ -34,6 +37,9 @@ import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
|
|
+import static org.mockito.Mockito.*;
|
|
|
+
|
|
|
/**
|
|
|
* This class tests the creation and validation of a checkpoint.
|
|
|
*/
|
|
@@ -150,4 +156,76 @@ public class TestSecurityTokenEditLog extends TestCase {
|
|
|
if(cluster != null) cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ public void testEditsForCancelOnTokenExpire() throws IOException,
|
|
|
+ InterruptedException {
|
|
|
+ long renewInterval = 2000;
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
|
|
|
+ conf.setLong(DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, renewInterval);
|
|
|
+ conf.setLong(DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, renewInterval*2);
|
|
|
+
|
|
|
+ Text renewer = new Text(UserGroupInformation.getCurrentUser().getUserName());
|
|
|
+ FSImage fsImage = mock(FSImage.class);
|
|
|
+ FSEditLog log = mock(FSEditLog.class);
|
|
|
+ doReturn(log).when(fsImage).getEditLog();
|
|
|
+ FSNamesystem fsn = new FSNamesystem(fsImage, conf);
|
|
|
+
|
|
|
+ DelegationTokenSecretManager dtsm = fsn.getDelegationTokenSecretManager();
|
|
|
+ try {
|
|
|
+ dtsm.startThreads();
|
|
|
+
|
|
|
+ // get two tokens
|
|
|
+ Token<DelegationTokenIdentifier> token1 = fsn.getDelegationToken(renewer);
|
|
|
+ Token<DelegationTokenIdentifier> token2 = fsn.getDelegationToken(renewer);
|
|
|
+ DelegationTokenIdentifier ident1 = decodeIdentifier(token1);
|
|
|
+ DelegationTokenIdentifier ident2 = decodeIdentifier(token2);
|
|
|
+
|
|
|
+ // verify we got the tokens
|
|
|
+ verify(log, times(1)).logGetDelegationToken(eq(ident1), anyLong());
|
|
|
+ verify(log, times(1)).logGetDelegationToken(eq(ident2), anyLong());
|
|
|
+
|
|
|
+ // this is a little tricky because DTSM doesn't let us set scan interval
|
|
|
+ // so need to periodically sleep, then stop/start threads to force scan
|
|
|
+
|
|
|
+ // renew first token 1/2 to expire
|
|
|
+ Thread.sleep(renewInterval/2);
|
|
|
+ fsn.renewDelegationToken(token2);
|
|
|
+ verify(log, times(1)).logRenewDelegationToken(eq(ident2), anyLong());
|
|
|
+ // force scan and give it a little time to complete
|
|
|
+ dtsm.stopThreads(); dtsm.startThreads();
|
|
|
+ Thread.sleep(250);
|
|
|
+ // no token has expired yet
|
|
|
+ verify(log, times(0)).logCancelDelegationToken(eq(ident1));
|
|
|
+ verify(log, times(0)).logCancelDelegationToken(eq(ident2));
|
|
|
+
|
|
|
+ // sleep past expiration of 1st non-renewed token
|
|
|
+ Thread.sleep(renewInterval/2);
|
|
|
+ dtsm.stopThreads(); dtsm.startThreads();
|
|
|
+ Thread.sleep(250);
|
|
|
+ // non-renewed token should have implicitly been cancelled
|
|
|
+ verify(log, times(1)).logCancelDelegationToken(eq(ident1));
|
|
|
+ verify(log, times(0)).logCancelDelegationToken(eq(ident2));
|
|
|
+
|
|
|
+ // sleep past expiration of 2nd renewed token
|
|
|
+ Thread.sleep(renewInterval/2);
|
|
|
+ dtsm.stopThreads(); dtsm.startThreads();
|
|
|
+ Thread.sleep(250);
|
|
|
+ // both tokens should have been implicitly cancelled by now
|
|
|
+ verify(log, times(1)).logCancelDelegationToken(eq(ident1));
|
|
|
+ verify(log, times(1)).logCancelDelegationToken(eq(ident2));
|
|
|
+ } finally {
|
|
|
+ dtsm.stopThreads();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static DelegationTokenIdentifier decodeIdentifier(Token<?> token)
|
|
|
+ throws IOException {
|
|
|
+ DelegationTokenIdentifier ident = new DelegationTokenIdentifier();
|
|
|
+ ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
|
|
|
+ DataInputStream in = new DataInputStream(buf);
|
|
|
+ ident.readFields(in);
|
|
|
+ in.close();
|
|
|
+ return ident;
|
|
|
+ }
|
|
|
}
|