diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index d180ea080c..7b48a4b588 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -368,6 +368,9 @@ Release 2.7.0 - UNRELEASED HDFS-7227. Fix findbugs warning about NP_DEREFERENCE_OF_READLINE_VALUE in SpanReceiverHost (cmccabe) + HADOOP-11170. ZKDelegationTokenSecretManager fails to renewToken created by + a peer. (Arun Suresh and Gregory Chanan via kasha) + Release 2.6.0 - UNRELEASED INCOMPATIBLE CHANGES diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java index ebba6a192a..ac399ec2cd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java @@ -20,10 +20,13 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.Text; import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.util.Arrays; import java.util.HashMap; @@ -125,7 +128,7 @@ public void startThreads() throws IOException { * Reset all data structures and mutable state. */ public synchronized void reset() { - currentId = 0; + setCurrentKeyId(0); allKeys.clear(); setDelegationTokenSeqNum(0); currentTokens.clear(); @@ -138,8 +141,8 @@ public synchronized void reset() { public synchronized void addKey(DelegationKey key) throws IOException { if (running) // a safety check throw new IOException("Can't add delegation key to a running SecretManager."); - if (key.getKeyId() > currentId) { - currentId = key.getKeyId(); + if (key.getKeyId() > getCurrentKeyId()) { + setCurrentKeyId(key.getKeyId()); } allKeys.put(key.getKeyId(), key); } @@ -182,6 +185,30 @@ protected void updateStoredToken(TokenIdent ident, long renewDate) throws IOExce return; } + /** + * For subclasses externalizing the storage, for example Zookeeper + * based implementations + */ + protected synchronized int getCurrentKeyId() { + return currentId; + } + + /** + * For subclasses externalizing the storage, for example Zookeeper + * based implementations + */ + protected synchronized int incrementCurrentKeyId() { + return ++currentId; + } + + /** + * For subclasses externalizing the storage, for example Zookeeper + * based implementations + */ + protected synchronized void setCurrentKeyId(int keyId) { + currentId = keyId; + } + /** * For subclasses externalizing the storage, for example Zookeeper * based implementations @@ -282,8 +309,8 @@ public synchronized void addPersistedDelegationToken( return; } byte[] password = createPassword(identifier.getBytes(), dKey.getKey()); - if (identifier.getSequenceNumber() > delegationTokenSequenceNumber) { - delegationTokenSequenceNumber = identifier.getSequenceNumber(); + if (identifier.getSequenceNumber() > getDelegationTokenSeqNum()) { + setDelegationTokenSeqNum(identifier.getSequenceNumber()); } if (getTokenInfo(identifier) == null) { currentTokens.put(identifier, new DelegationTokenInformation(renewDate, @@ -303,7 +330,7 @@ private void updateCurrentKey() throws IOException { /* Create a new currentKey with an estimated expiry date. */ int newCurrentId; synchronized (this) { - newCurrentId = currentId+1; + newCurrentId = incrementCurrentKeyId(); } DelegationKey newKey = new DelegationKey(newCurrentId, System .currentTimeMillis() @@ -311,7 +338,6 @@ private void updateCurrentKey() throws IOException { //Log must be invoked outside the lock on 'this' logUpdateMasterKey(newKey); synchronized (this) { - currentId = newKey.getKeyId(); currentKey = newKey; storeDelegationKey(currentKey); } @@ -358,9 +384,9 @@ protected synchronized byte[] createPassword(TokenIdent identifier) { sequenceNum = incrementDelegationTokenSeqNum(); identifier.setIssueDate(now); identifier.setMaxDate(now + tokenMaxLifetime); - identifier.setMasterKeyId(currentId); + identifier.setMasterKeyId(currentKey.getKeyId()); identifier.setSequenceNumber(sequenceNum); - LOG.info("Creating password for identifier: " + identifier); + LOG.info("Creating password for identifier: [" + MD5Hash.digest(identifier.getBytes()) + ", " + currentKey.getKeyId() + "]"); byte[] password = createPassword(identifier.getBytes(), currentKey.getKey()); DelegationTokenInformation tokenInfo = new DelegationTokenInformation(now + tokenRenewInterval, password, getTrackingIdIfEnabled(identifier)); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java index 5f688447b3..82dd2da7e1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java @@ -48,7 +48,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -56,6 +55,7 @@ import org.apache.zookeeper.client.ZooKeeperSaslClient; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,6 +104,7 @@ public abstract class ZKDelegationTokenSecretManager token = - (Token) tm1.createToken( - UserGroupInformation.getCurrentUser(), "foo"); - Assert.assertNotNull(token); - tm2.verifyToken(token); - - token = (Token) tm2.createToken( - UserGroupInformation.getCurrentUser(), "bar"); - Assert.assertNotNull(token); - tm1.verifyToken(token); - } finally { + @After + public void tearDown() throws Exception { + if (zkServer != null) { zkServer.close(); } } + + protected Configuration getSecretConf(String connectString) { + Configuration conf = new Configuration(); + conf.setBoolean(DelegationTokenManager.ENABLE_ZK_KEY, true); + conf.set(ZKDelegationTokenSecretManager.ZK_DTSM_ZK_CONNECTION_STRING, connectString); + conf.set(ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH, "testPath"); + conf.set(ZKDelegationTokenSecretManager.ZK_DTSM_ZK_AUTH_TYPE, "none"); + conf.setLong(DelegationTokenManager.UPDATE_INTERVAL, DAY_IN_SECS); + conf.setLong(DelegationTokenManager.MAX_LIFETIME, DAY_IN_SECS); + conf.setLong(DelegationTokenManager.RENEW_INTERVAL, DAY_IN_SECS); + conf.setLong(DelegationTokenManager.REMOVAL_SCAN_INTERVAL, DAY_IN_SECS); + return conf; + } + + @SuppressWarnings("unchecked") + @Test + public void testMultiNodeOperations() throws Exception { + DelegationTokenManager tm1, tm2 = null; + String connectString = zkServer.getConnectString(); + Configuration conf = getSecretConf(connectString); + tm1 = new DelegationTokenManager(conf, new Text("bla")); + tm1.init(); + tm2 = new DelegationTokenManager(conf, new Text("bla")); + tm2.init(); + + Token token = + (Token) tm1.createToken( + UserGroupInformation.getCurrentUser(), "foo"); + Assert.assertNotNull(token); + tm2.verifyToken(token); + tm2.renewToken(token, "foo"); + tm1.verifyToken(token); + tm1.cancelToken(token, "foo"); + try { + tm2.verifyToken(token); + fail("Expected InvalidToken"); + } catch (SecretManager.InvalidToken it) { + // Ignore + } + + token = (Token) tm2.createToken( + UserGroupInformation.getCurrentUser(), "bar"); + Assert.assertNotNull(token); + tm1.verifyToken(token); + tm1.renewToken(token, "bar"); + tm2.verifyToken(token); + tm2.cancelToken(token, "bar"); + try { + tm1.verifyToken(token); + fail("Expected InvalidToken"); + } catch (SecretManager.InvalidToken it) { + // Ignore + } + } + + @SuppressWarnings("unchecked") + @Test + public void testRenewTokenSingleManager() throws Exception { + DelegationTokenManager tm1 = null; + String connectString = zkServer.getConnectString(); + Configuration conf = getSecretConf(connectString); + tm1 = new DelegationTokenManager(conf, new Text("foo")); + tm1.init(); + + Token token = + (Token) + tm1.createToken(UserGroupInformation.getCurrentUser(), "foo"); + Assert.assertNotNull(token); + tm1.renewToken(token, "foo"); + tm1.verifyToken(token); + } + + @SuppressWarnings("unchecked") + @Test + public void testCancelTokenSingleManager() throws Exception { + DelegationTokenManager tm1 = null; + String connectString = zkServer.getConnectString(); + Configuration conf = getSecretConf(connectString); + tm1 = new DelegationTokenManager(conf, new Text("foo")); + tm1.init(); + + Token token = + (Token) + tm1.createToken(UserGroupInformation.getCurrentUser(), "foo"); + Assert.assertNotNull(token); + tm1.cancelToken(token, "foo"); + try { + tm1.verifyToken(token); + fail("Expected InvalidToken"); + } catch (SecretManager.InvalidToken it) { + it.printStackTrace(); + } + } }