Browse Source

HDFS-3841. Port HDFS-3835 to branch-0.23 (bobby via daryn)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1376485 13f79535-47bb-0310-9956-ffa450edef68
Daryn Sharp 12 years ago
parent
commit
1d489852ea

+ 11 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java

@@ -103,6 +103,16 @@ extends AbstractDelegationTokenIdentifier>
     }
     }
   }
   }
   
   
+  /**
+   * Reset all data structures and mutable state.
+   */
+  public synchronized void reset() {
+    currentId = 0;
+    allKeys.clear();
+    delegationTokenSequenceNumber = 0;
+    currentTokens.clear();
+  }
+  
   /** 
   /** 
    * Add a previously used master key to cache (when NN restarts), 
    * Add a previously used master key to cache (when NN restarts), 
    * should be called before activate().
    * should be called before activate().
@@ -180,7 +190,6 @@ extends AbstractDelegationTokenIdentifier>
   
   
   @Override
   @Override
   protected synchronized byte[] createPassword(TokenIdent identifier) {
   protected synchronized byte[] createPassword(TokenIdent identifier) {
-    LOG.info("Creating password for identifier: "+identifier);
     int sequenceNum;
     int sequenceNum;
     long now = System.currentTimeMillis();
     long now = System.currentTimeMillis();
     sequenceNum = ++delegationTokenSequenceNumber;
     sequenceNum = ++delegationTokenSequenceNumber;
@@ -188,6 +197,7 @@ extends AbstractDelegationTokenIdentifier>
     identifier.setMaxDate(now + tokenMaxLifetime);
     identifier.setMaxDate(now + tokenMaxLifetime);
     identifier.setMasterKeyId(currentId);
     identifier.setMasterKeyId(currentId);
     identifier.setSequenceNumber(sequenceNum);
     identifier.setSequenceNumber(sequenceNum);
+    LOG.info("Creating password for identifier: " + identifier);
     byte[] password = createPassword(identifier.getBytes(), currentKey.getKey());
     byte[] password = createPassword(identifier.getBytes(), currentKey.getKey());
     currentTokens.put(identifier, new DelegationTokenInformation(now
     currentTokens.put(identifier, new DelegationTokenInformation(now
         + tokenRenewInterval, password));
         + tokenRenewInterval, password));

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -108,6 +108,8 @@ Release 0.23.3 - UNRELEASED
     HDFS-3788. ByteRangeInputStream should not expect HTTP Content-Length header
     HDFS-3788. ByteRangeInputStream should not expect HTTP Content-Length header
     when chunked transfer-encoding is used.  (szetszwo)
     when chunked transfer-encoding is used.  (szetszwo)
 
 
+    HDFS-3841. Port HDFS-3835 to branch-0.23 (bobby via daryn)
+
 Release 0.23.2 - UNRELEASED
 Release 0.23.2 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -560,6 +561,10 @@ public class FSImage implements Closeable {
    */
    */
   void reloadFromImageFile(File file) throws IOException {
   void reloadFromImageFile(File file) throws IOException {
     namesystem.dir.reset();
     namesystem.dir.reset();
+    DelegationTokenSecretManager man = namesystem.getDelegationTokenSecretManager();
+    if (man != null) {
+      man.reset();
+    }
 
 
     LOG.debug("Reloading namespace from " + file);
     LOG.debug("Reloading namespace from " + file);
     loadFSImage(file);
     loadFSImage(file);

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -433,6 +433,7 @@ public class SecondaryNameNode implements Runnable {
               downloadImage = false;
               downloadImage = false;
               LOG.info("Image has not changed. Will not download image.");
               LOG.info("Image has not changed. Will not download image.");
             } else {
             } else {
+              LOG.info("Image has changed. Downloading updated image from NN.");
               MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
               MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
                   nnHostPort, sig.mostRecentCheckpointTxId, dstImage.getStorage(), true);
                   nnHostPort, sig.mostRecentCheckpointTxId, dstImage.getStorage(), true);
               dstImage.saveDigestAndRenameCheckpointImage(
               dstImage.saveDigestAndRenameCheckpointImage(