Ver Fonte

Merging trunk to branch-trunk-win

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-trunk-win@1420493 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas há 12 anos atrás
pai
commit
c1fe43311f

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -482,6 +482,9 @@ Release 2.0.3-alpha - Unreleased
     HADOOP-9126. FormatZK and ZKFC startup can fail due to zkclient connection
     establishment delay. (Rakesh R and todd via todd)
 
+    HADOOP-9113. o.a.h.fs.TestDelegationTokenRenewer is failing intermittently.
+    (Karthik Kambatla via eli)
+
 Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.fs;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.IOException;
 import java.lang.ref.WeakReference;
 import java.util.concurrent.DelayQueue;
@@ -147,6 +149,12 @@ public class DelegationTokenRenewer
   /** Queue to maintain the RenewActions to be processed by the {@link #run()} */
   private volatile DelayQueue<RenewAction<?>> queue = new DelayQueue<RenewAction<?>>();
   
+  /** For testing purposes */
+  @VisibleForTesting
+  protected int getRenewQueueLength() {
+    return queue.size();
+  }
+
   /**
    * Create the singleton instance. However, the thread can be started lazily in
    * {@link #addRenewAction(FileSystem)}

+ 5 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDelegationTokenRenewer.java

@@ -4,6 +4,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 
@@ -133,6 +134,8 @@ public class TestDelegationTokenRenewer {
       InterruptedException {
     TestFileSystem tfs = new TestFileSystem();
     renewer.addRenewAction(tfs);
+    assertEquals("FileSystem not added to DelegationTokenRenewer", 1,
+        renewer.getRenewQueueLength());
 
     for (int i = 0; i < 60; i++) {
       Thread.sleep(RENEW_CYCLE);
@@ -144,7 +147,8 @@ public class TestDelegationTokenRenewer {
 
     assertTrue("Token not renewed even after 1 minute",
         (tfs.testToken.renewCount > 0));
-    assertTrue("Token not removed", (tfs.testToken.renewCount < MAX_RENEWALS));
+    assertEquals("FileSystem not removed from DelegationTokenRenewer", 0,
+        renewer.getRenewQueueLength());
     assertTrue("Token not cancelled", tfs.testToken.cancelled);
   }
 }

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

@@ -269,6 +269,9 @@ Trunk (Unreleased)
     HDFS-4260 Fix HDFS tests to set test dir to a valid HDFS path as opposed
     to the local build path (Chri Nauroth via Sanjay)
 
+    HDFS-4269. Datanode rejects all datanode registrations from localhost
+    in single-node developer setup on Windows. (Chris Nauroth via suresh)
+
 Release 2.0.3-alpha - Unreleased 
 
   INCOMPATIBLE CHANGES

+ 19 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -633,7 +633,9 @@ public class DatanodeManager {
       // Mostly called inside an RPC, update ip and peer hostname
       String hostname = dnAddress.getHostName();
       String ip = dnAddress.getHostAddress();
-      if (hostname.equals(ip)) {
+      if (!isNameResolved(dnAddress)) {
+        // Reject registration of unresolved datanode to prevent performance
+        // impact of repetitive DNS lookups later.
         LOG.warn("Unresolved datanode registration from " + ip);
         throw new DisallowedDatanodeException(nodeReg);
       }
@@ -1061,6 +1063,22 @@ public class DatanodeManager {
     }
     return names;
   }
+
+  /**
+   * Checks if name resolution was successful for the given address.  If IP
+   * address and host name are the same, then it means name resolution has
+   * failed.  As a special case, the loopback address is also considered
+   * acceptable.  This is particularly important on Windows, where 127.0.0.1 does
+   * not resolve to "localhost".
+   * 
+   * @param address InetAddress to check
+   * @return boolean true if name resolution successful or address is loopback
+   */
+  private static boolean isNameResolved(InetAddress address) {
+    String hostname = address.getHostName();
+    String ip = address.getHostAddress();
+    return !hostname.equals(ip) || address.isLoopbackAddress();
+  }
   
   private void setDatanodeDead(DatanodeDescriptor node) {
     node.setLastUpdate(0);