Browse Source

HDFS-5338. Add a conf to disable hostname check in datanode registration.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1532468 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 years ago
parent
commit
0a5f0efcb4

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

@@ -313,6 +313,9 @@ Release 2.3.0 - UNRELEASED
 
     HDFS-5323.  Remove some deadcode in BlockManager (Colin Patrick McCabe)
 
+    HDFS-5338. Add a conf to disable hostname check in datanode registration.
+    (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -193,7 +193,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_DATANODE_SYNCONCLOSE_DEFAULT = false;
   public static final String  DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY = "dfs.datanode.socket.reuse.keepalive";
   public static final int     DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT = 1000;
-  
+
+  public static final String DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY = "dfs.namenode.datanode.registration.ip-hostname-check";
+  public static final boolean DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT = true;
+
   // Whether to enable datanode's stale state detection and usage for reads
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
   public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT = false;

+ 15 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -142,6 +142,7 @@ public class DatanodeManager {
    */
   private boolean hasClusterEverBeenMultiRack = false;
 
+  private final boolean checkIpHostnameInRegistration;
   /**
    * The number of datanodes for each software version. This list should change
    * during rolling upgrades.
@@ -210,6 +211,12 @@ public class DatanodeManager {
     LOG.info(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY
         + "=" + this.blockInvalidateLimit);
 
+    this.checkIpHostnameInRegistration = conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY,
+        DFSConfigKeys.DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT);
+    LOG.info(DFSConfigKeys.DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY
+        + "=" + checkIpHostnameInRegistration);
+
     this.avoidStaleDataNodesForRead = conf.getBoolean(
         DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY,
         DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT);
@@ -733,11 +740,13 @@ public class DatanodeManager {
       // Mostly called inside an RPC, update ip and peer hostname
       String hostname = dnAddress.getHostName();
       String ip = dnAddress.getHostAddress();
-      if (!isNameResolved(dnAddress)) {
+      if (checkIpHostnameInRegistration && !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);
+        final String message = "hostname cannot be resolved (ip="
+            + ip + ", hostname=" + hostname + ")";
+        LOG.warn("Unresolved datanode registration: " + message);
+        throw new DisallowedDatanodeException(nodeReg, message);
       }
       // update node registration with the ip and hostname from rpc request
       nodeReg.setIpAddr(ip);
@@ -1185,17 +1194,17 @@ public class DatanodeManager {
   /**
    * 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
+   * failed.  As a special case, local addresses are 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
+   * @return boolean true if name resolution successful or address is local
    */
   private static boolean isNameResolved(InetAddress address) {
     String hostname = address.getHostName();
     String ip = address.getHostAddress();
-    return !hostname.equals(ip) || address.isLoopbackAddress();
+    return !hostname.equals(ip) || NetUtils.isLocalAddress(address);
   }
   
   private void setDatanodeDead(DatanodeDescriptor node) {

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DisallowedDatanodeException.java

@@ -37,7 +37,12 @@ public class DisallowedDatanodeException extends IOException {
   /** for java.io.Serializable */
   private static final long serialVersionUID = 1L;
 
+  public DisallowedDatanodeException(DatanodeID nodeID, String reason) {
+    super("Datanode denied communication with namenode because "
+        + reason + ": " + nodeID);
+  }
+
   public DisallowedDatanodeException(DatanodeID nodeID) {
-    super("Datanode denied communication with namenode: " + nodeID);
+    this(nodeID, "the host is not in the include-list");
   }
 }

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -569,6 +569,22 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.datanode.registration.ip-hostname-check</name>
+  <value>true</value>
+  <description>
+    If true (the default), then the namenode requires that a connecting
+    datanode's address must be resolved to a hostname.  If necessary, a reverse
+    DNS lookup is performed.  All attempts to register a datanode from an
+    unresolvable address are rejected.
+
+    It is recommended that this setting be left on to prevent accidental
+    registration of datanodes listed by hostname in the excludes file during a
+    DNS outage.  Only set this to false in environments where there is no
+    infrastructure to support reverse DNS lookup.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.decommission.interval</name>
   <value>30</value>