You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cd...@apache.org on 2009/05/05 04:22:48 UTC

svn commit: r771525 - in /hadoop/core/trunk: CHANGES.txt src/core/org/apache/hadoop/net/DNS.java src/test/org/apache/hadoop/net/TestDNS.java

Author: cdouglas
Date: Tue May  5 02:22:47 2009
New Revision: 771525

URL: http://svn.apache.org/viewvc?rev=771525&view=rev
Log:
HADOOP-3426. Fix/provide handling when DNS lookup fails on the loopback
address. Also cache the result of the lookup. Contributed by Steve Loughran

Added:
    hadoop/core/trunk/src/test/org/apache/hadoop/net/TestDNS.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/core/org/apache/hadoop/net/DNS.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=771525&r1=771524&r2=771525&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Tue May  5 02:22:47 2009
@@ -508,6 +508,9 @@
     HADOOP-5406. Fix JNI binding for ZlibCompressor::setDictionary. (Lars
     Francke via cdouglas)
 
+    HADOOP-3426. Fix/provide handling when DNS lookup fails on the loopback
+    address. Also cache the result of the lookup. (Steve Loughran via cdouglas)
+
 Release 0.20.1 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/net/DNS.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/net/DNS.java?rev=771525&r1=771524&r2=771525&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/net/DNS.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/net/DNS.java Tue May  5 02:22:47 2009
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.net;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 import java.net.InetAddress;
 import java.net.NetworkInterface;
 import java.net.SocketException;
@@ -39,17 +42,25 @@
  */
 public class DNS {
 
+  private static final Log LOG = LogFactory.getLog(DNS.class);
+
+  /**
+   * The cached hostname -initially null.
+   */
+
+  private static final String cachedHostname = resolveLocalHostname();
+  private static final String cachedHostAddress = resolveLocalHostIPAddress();
+  private static final String LOCALHOST = "localhost";
+
   /**
    * Returns the hostname associated with the specified IP address by the
    * provided nameserver.
-   * 
-   * @param hostIp
-   *            The address to reverse lookup
-   * @param ns
-   *            The host name of a reachable DNS server
+   *
+   * Loopback addresses 
+   * @param hostIp The address to reverse lookup
+   * @param ns The host name of a reachable DNS server
    * @return The host name associated with the provided IP
-   * @throws NamingException
-   *             If a NamingException is encountered
+   * @throws NamingException If a NamingException is encountered
    */
   public static String reverseDns(InetAddress hostIp, String ns)
     throws NamingException {
@@ -62,13 +73,16 @@
       + parts[0] + ".in-addr.arpa";
 
     DirContext ictx = new InitialDirContext();
-    Attributes attribute =
-      ictx.getAttributes("dns://"               // Use "dns:///" if the default
-                         + ((ns == null) ? "" : ns) + 
+    Attributes attribute;
+    try {
+      attribute = ictx.getAttributes("dns://"               // Use "dns:///" if the default
+                         + ((ns == null) ? "" : ns) +
                          // nameserver is to be used
                          "/" + reverseIP, new String[] { "PTR" });
-    ictx.close();
-    
+    } finally {
+      ictx.close();
+    }
+
     return attribute.get("PTR").get().toString();
   }
 
@@ -89,25 +103,26 @@
     throws UnknownHostException {
     try {
       NetworkInterface netIF = NetworkInterface.getByName(strInterface);
-      if (netIF == null)
-        return new String[] { InetAddress.getLocalHost()
-                              .getHostAddress() };
-      else {
+      if (netIF == null) {
+        return new String[] { cachedHostAddress };
+      } else {
         Vector<String> ips = new Vector<String>();
         Enumeration e = netIF.getInetAddresses();
-        while (e.hasMoreElements())
+        while (e.hasMoreElements()) {
           ips.add(((InetAddress) e.nextElement()).getHostAddress());
+        }
         return ips.toArray(new String[] {});
       }
     } catch (SocketException e) {
-      return new String[] { InetAddress.getLocalHost().getHostAddress() };
+      return new String[]  { cachedHostAddress };
     }
   }
 
-  /**
+
+    /**
    * Returns the first available IP address associated with the provided
    * network interface
-   * 
+   *
    * @param strInterface
    *            The name of the network interface to query (e.g. eth0)
    * @return The IP address in text form
@@ -123,14 +138,14 @@
   /**
    * Returns all the host names associated by the provided nameserver with the
    * address bound to the specified network interface
-   * 
+   *
    * @param strInterface
    *            The name of the network interface to query (e.g. eth0)
    * @param nameserver
    *            The DNS host name
    * @return A string vector of all host names associated with the IPs tied to
    *         the specified interface
-   * @throws UnknownHostException
+   * @throws UnknownHostException if the hostname cannot be determined
    */
   public static String[] getHosts(String strInterface, String nameserver)
     throws UnknownHostException {
@@ -140,16 +155,67 @@
       try {
         hosts.add(reverseDns(InetAddress.getByName(ips[ctr]),
                              nameserver));
-      } catch (Exception e) {
+      } catch (UnknownHostException ignored) {
+      } catch (NamingException ignored) {
       }
 
-    if (hosts.size() == 0)
-      return new String[] { InetAddress.getLocalHost().getCanonicalHostName() };
-    else
-      return hosts.toArray(new String[] {});
+    if (hosts.isEmpty()) {
+      return new String[] { cachedHostname };
+    } else {
+      return hosts.toArray(new String[hosts.size()]);
+    }
+  }
+
+
+  /**
+   * Determine the local hostname; retrieving it from cache if it is known
+   * If we cannot determine our host name, return "localhost"
+   * @return the local hostname or "localhost"
+   */
+  private static String resolveLocalHostname() {
+    String localhost;
+    try {
+      localhost = InetAddress.getLocalHost().getCanonicalHostName();
+    } catch (UnknownHostException e) {
+      LOG.info("Unable to determine local hostname "
+              + "-falling back to \"" + LOCALHOST + "\"", e);
+      localhost = LOCALHOST;
+    }
+    return localhost;
   }
 
+
   /**
+   * Get the IPAddress of the local host as a string.
+   * This will be a loop back value if the local host address cannot be
+   * determined.
+   * If the loopback address of "localhost" does not resolve, then the system's
+   * network is in such a state that nothing is going to work. A message is
+   * logged at the error level and a null pointer returned, a pointer
+   * which will trigger failures later on the application
+   * @return the IPAddress of the local host or null for a serious problem.
+   */
+  private static String resolveLocalHostIPAddress() {
+    String address;
+      try {
+        address = InetAddress.getLocalHost().getHostAddress();
+      } catch (UnknownHostException e) {
+        LOG.info("Unable to determine address of the host"
+                + "-falling back to \"" + LOCALHOST + "\" address", e);
+        try {
+          address = InetAddress.getByName(LOCALHOST).getHostAddress();
+        } catch (UnknownHostException noLocalHostAddressException) {
+          //at this point, deep trouble
+          LOG.error("Unable to determine local loopback address "
+                  + "of \"" + LOCALHOST + "\" " +
+                  "-this system's network configuration is unsupported", e);
+          address = null;
+        }
+      }
+    return address;
+  }
+
+    /**
    * Returns all the host names associated by the default nameserver with the
    * address bound to the specified network interface
    * 
@@ -158,7 +224,7 @@
    * @return The list of host names associated with IPs bound to the network
    *         interface
    * @throws UnknownHostException
-   *             If one is encountered while querying the deault interface
+   *             If one is encountered while querying the default interface
    * 
    */
   public static String[] getHosts(String strInterface)
@@ -177,15 +243,17 @@
    * @return The default host names associated with IPs bound to the network
    *         interface
    * @throws UnknownHostException
-   *             If one is encountered while querying the deault interface
+   *             If one is encountered while querying the default interface
    */
   public static String getDefaultHost(String strInterface, String nameserver)
     throws UnknownHostException {
-    if (strInterface.equals("default")) 
-      return InetAddress.getLocalHost().getCanonicalHostName();
+    if ("default".equals(strInterface)) {
+      return cachedHostname;
+    }
 
-    if (nameserver != null && nameserver.equals("default"))
+    if ("default".equals(nameserver)) {
       return getDefaultHost(strInterface);
+    }
 
     String[] hosts = getHosts(strInterface, nameserver);
     return hosts[0];
@@ -196,11 +264,12 @@
    * nameserver with the address bound to the specified network interface
    * 
    * @param strInterface
-   *            The name of the network interface to query (e.g. eth0)
+   *            The name of the network interface to query (e.g. eth0).
+   *            Must not be null.
    * @return The default host name associated with IPs bound to the network
    *         interface
    * @throws UnknownHostException
-   *             If one is encountered while querying the deault interface
+   *             If one is encountered while querying the default interface
    */
   public static String getDefaultHost(String strInterface)
     throws UnknownHostException {

Added: hadoop/core/trunk/src/test/org/apache/hadoop/net/TestDNS.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/net/TestDNS.java?rev=771525&view=auto
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/net/TestDNS.java (added)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/net/TestDNS.java Tue May  5 02:22:47 2009
@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.net;
+
+import junit.framework.TestCase;
+
+import java.net.UnknownHostException;
+import java.net.InetAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import javax.naming.NameNotFoundException;
+
+/**
+ *
+ */
+public class TestDNS extends TestCase {
+
+  private static final Log LOG = LogFactory.getLog(TestDNS.class);
+  private static final String DEFAULT = "default";
+
+  /**
+   * Constructs a test case with the given name.
+   *
+   * @param name test name
+   */
+  public TestDNS(String name) {
+    super(name);
+  }
+
+  /**
+   * Test that asking for the default hostname works
+   * @throws Exception if hostname lookups fail   */
+  public void testGetLocalHost() throws Exception {
+    String hostname = DNS.getDefaultHost(DEFAULT);
+    assertNotNull(hostname);
+  }
+
+  /**
+   * Test that repeated calls to getting the local host are fairly fast, and
+   * hence that caching is being used
+   * @throws Exception if hostname lookups fail
+   */
+  public void testGetLocalHostIsFast() throws Exception {
+    String hostname = DNS.getDefaultHost(DEFAULT);
+    assertNotNull(hostname);
+    long t1 = System.currentTimeMillis();
+    String hostname2 = DNS.getDefaultHost(DEFAULT);
+    long t2 = System.currentTimeMillis();
+    String hostname3 = DNS.getDefaultHost(DEFAULT);
+    long t3 = System.currentTimeMillis();
+    assertEquals(hostname3, hostname2);
+    assertEquals(hostname2, hostname);
+    long interval2 = t3 - t2;
+    assertTrue(
+            "It is taking to long to determine the local host -caching is not working",
+            interval2 < 20000);
+  }
+
+  /**
+   * Test that our local IP address is not null
+   * @throws Exception if something went wrong
+   */
+  public void testLocalHostHasAnAddress() throws Exception {
+    assertNotNull(getLocalIPAddr());
+  }
+
+  private InetAddress getLocalIPAddr() throws UnknownHostException {
+    String hostname = DNS.getDefaultHost(DEFAULT);
+    InetAddress localhost = InetAddress.getByName(hostname);
+    return localhost;
+  }
+
+  /**
+   * Test that passing a null pointer is as the interface
+   * fails with a NullPointerException
+   * @throws Exception if something went wrong
+   */
+  public void testNullInterface() throws Exception {
+    try {
+      String host = DNS.getDefaultHost(null);
+      fail("Expected a NullPointerException, got " + host);
+    } catch (NullPointerException expected) {
+      //this is expected
+    }
+  }
+
+  /**
+   * Get the IP addresses of an unknown interface, expect to get something
+   * back
+   * @throws Exception if something went wrong
+   */
+  public void testIPsOfUnknownInterface() throws Exception {
+    String[] ips = DNS.getIPs("name-of-an-unknown-interface");
+    assertNotNull(ips);
+    assertTrue(ips.length > 0);
+  }
+
+  /**
+   * TestCase: get our local address and reverse look it up
+   * @throws Exception if that fails
+   */
+  public void testRDNS() throws Exception {
+    InetAddress localhost = getLocalIPAddr();
+    try {
+      String s = DNS.reverseDns(localhost, null);
+      LOG.info("Local revers DNS hostname is " + s);
+    } catch (NameNotFoundException e) {
+      if (!localhost.isLinkLocalAddress() || localhost.isLoopbackAddress()) {
+        //these addresses probably won't work with rDNS anyway, unless someone
+        //has unusual entries in their DNS server mapping 1.0.0.127 to localhost
+        LOG.info("Reverse DNS failing as due to incomplete networking", e);
+        LOG.info("Address is " + localhost
+                + " Loopback=" + localhost.isLoopbackAddress()
+                + " Linklocal=" + localhost.isLinkLocalAddress());
+      }
+
+    }
+  }
+
+  /**
+   * Test that the name "localhost" resolves to something.
+   *
+   * If this fails, your machine's network is in a mess, go edit /etc/hosts
+   * @throws Exception for any problems
+   */
+  public void testLocalhostResolves() throws Exception {
+    InetAddress localhost = InetAddress.getByName("localhost");
+    assertNotNull("localhost is null", localhost);
+    LOG.info("Localhost IPAddr is " + localhost.toString());
+  }
+}