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 su...@apache.org on 2012/01/11 22:02:05 UTC

svn commit: r1230249 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/net/ src/main/java/org/apache/hadoop/security/ src/test/java/org/apache/hadoop/fs/ src/test/java/org/apache/hadoop/net/ src/test/java...

Author: suresh
Date: Wed Jan 11 21:02:04 2012
New Revision: 1230249

URL: http://svn.apache.org/viewvc?rev=1230249&view=rev
Log:
HADOOP-7964. Deadlock in NetUtils and SecurityUtil class initialization. Contributed by Daryn Sharp.

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/NetUtilsTestResolver.java
      - copied, changed from r1229743, hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/NetUtilsTestResolver.java
Removed:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/NetUtilsTestResolver.java
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCanonicalization.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1230249&r1=1230248&r2=1230249&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Wed Jan 11 21:02:04 2012
@@ -266,6 +266,9 @@ Release 0.23.1 - Unreleased
    HADOOP-7963. Fix ViewFS to catch a null canonical service-name and pass
    tests TestViewFileSystem* (Siddharth Seth via vinodkv)
 
+   HADOOP-7964. Deadlock in NetUtils and SecurityUtil class initialization.
+   (Daryn Sharp via suresh)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java?rev=1230249&r1=1230248&r2=1230249&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java Wed Jan 11 21:02:04 2012
@@ -51,12 +51,6 @@ import org.apache.hadoop.ipc.VersionedPr
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import com.google.common.annotations.VisibleForTesting;
-
-//this will need to be replaced someday when there is a suitable replacement
-import sun.net.dns.ResolverConfiguration;
-import sun.net.util.IPAddressUtil;
-
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Unstable
 public class NetUtils {
@@ -72,26 +66,6 @@ public class NetUtils {
   /** Base URL of the Hadoop Wiki: {@value} */
   public static final String HADOOP_WIKI = "http://wiki.apache.org/hadoop/";
 
-  private static HostResolver hostResolver;
-  
-  static {
-    // SecurityUtils requires a more secure host resolver if tokens are
-    // using hostnames
-    setUseQualifiedHostResolver(!SecurityUtil.getTokenServiceUseIp());
-  }
-
-  /**
-   * This method is intended for use only by SecurityUtils!
-   * @param flag where the qualified or standard host resolver is used
-   *             to create socket addresses
-   */
-  @InterfaceAudience.Private
-  public static void setUseQualifiedHostResolver(boolean flag) {
-      hostResolver = flag
-          ? new QualifiedHostResolver()
-          : new StandardHostResolver();
-  }
-  
   /**
    * Get the socket factory for the given class according to its
    * configuration parameter
@@ -249,7 +223,7 @@ public class NetUtils {
     
     InetSocketAddress addr;
     try {
-      InetAddress iaddr = hostResolver.getByName(resolveHost);
+      InetAddress iaddr = SecurityUtil.getByName(resolveHost);
       // if there is a static entry for the host, make the returned
       // address look like the original given host
       if (staticHost != null) {
@@ -261,151 +235,6 @@ public class NetUtils {
     }
     return addr;
   }
-
-  interface HostResolver {
-    InetAddress getByName(String host) throws UnknownHostException;    
-  }
-  
-  /**
-   * Uses standard java host resolution
-   */
-  static class StandardHostResolver implements HostResolver {
-    public InetAddress getByName(String host) throws UnknownHostException {
-      return InetAddress.getByName(host);
-    }
-  }
-  
-  /**
-   * This an alternate resolver with important properties that the standard
-   * java resolver lacks:
-   * 1) The hostname is fully qualified.  This avoids security issues if not
-   *    all hosts in the cluster do not share the same search domains.  It
-   *    also prevents other hosts from performing unnecessary dns searches.
-   *    In contrast, InetAddress simply returns the host as given.
-   * 2) The InetAddress is instantiated with an exact host and IP to prevent
-   *    further unnecessary lookups.  InetAddress may perform an unnecessary
-   *    reverse lookup for an IP.
-   * 3) A call to getHostName() will always return the qualified hostname, or
-   *    more importantly, the IP if instantiated with an IP.  This avoids
-   *    unnecessary dns timeouts if the host is not resolvable.
-   * 4) Point 3 also ensures that if the host is re-resolved, ex. during a
-   *    connection re-attempt, that a reverse lookup to host and forward
-   *    lookup to IP is not performed since the reverse/forward mappings may
-   *    not always return the same IP.  If the client initiated a connection
-   *    with an IP, then that IP is all that should ever be contacted.
-   *    
-   * NOTE: this resolver is only used if:
-   *       hadoop.security.token.service.use_ip=false 
-   */
-  protected static class QualifiedHostResolver implements HostResolver {
-    @SuppressWarnings("unchecked")
-    private List<String> searchDomains =
-        ResolverConfiguration.open().searchlist();
-    
-    /**
-     * Create an InetAddress with a fully qualified hostname of the given
-     * hostname.  InetAddress does not qualify an incomplete hostname that
-     * is resolved via the domain search list.
-     * {@link InetAddress#getCanonicalHostName()} will fully qualify the
-     * hostname, but it always return the A record whereas the given hostname
-     * may be a CNAME.
-     * 
-     * @param host a hostname or ip address
-     * @return InetAddress with the fully qualified hostname or ip
-     * @throws UnknownHostException if host does not exist
-     */
-    public InetAddress getByName(String host) throws UnknownHostException {
-      InetAddress addr = null;
-
-      if (IPAddressUtil.isIPv4LiteralAddress(host)) {
-        // use ipv4 address as-is
-        byte[] ip = IPAddressUtil.textToNumericFormatV4(host);
-        addr = InetAddress.getByAddress(host, ip);
-      } else if (IPAddressUtil.isIPv6LiteralAddress(host)) {
-        // use ipv6 address as-is
-        byte[] ip = IPAddressUtil.textToNumericFormatV6(host);
-        addr = InetAddress.getByAddress(host, ip);
-      } else if (host.endsWith(".")) {
-        // a rooted host ends with a dot, ex. "host."
-        // rooted hosts never use the search path, so only try an exact lookup
-        addr = getByExactName(host);
-      } else if (host.contains(".")) {
-        // the host contains a dot (domain), ex. "host.domain"
-        // try an exact host lookup, then fallback to search list
-        addr = getByExactName(host);
-        if (addr == null) {
-          addr = getByNameWithSearch(host);
-        }
-      } else {
-        // it's a simple host with no dots, ex. "host"
-        // try the search list, then fallback to exact host
-        InetAddress loopback = InetAddress.getByName(null);
-        if (host.equalsIgnoreCase(loopback.getHostName())) {
-          addr = InetAddress.getByAddress(host, loopback.getAddress());
-        } else {
-          addr = getByNameWithSearch(host);
-          if (addr == null) {
-            addr = getByExactName(host);
-          }
-        }
-      }
-      // unresolvable!
-      if (addr == null) {
-        throw new UnknownHostException(host);
-      }
-      return addr;
-    }
-
-    InetAddress getByExactName(String host) {
-      InetAddress addr = null;
-      // InetAddress will use the search list unless the host is rooted
-      // with a trailing dot.  The trailing dot will disable any use of the
-      // search path in a lower level resolver.  See RFC 1535.
-      String fqHost = host;
-      if (!fqHost.endsWith(".")) fqHost += ".";
-      try {
-        addr = getInetAddressByName(fqHost);
-        // can't leave the hostname as rooted or other parts of the system
-        // malfunction, ex. kerberos principals are lacking proper host
-        // equivalence for rooted/non-rooted hostnames
-        addr = InetAddress.getByAddress(host, addr.getAddress());
-      } catch (UnknownHostException e) {
-        // ignore, caller will throw if necessary
-      }
-      return addr;
-    }
-
-    InetAddress getByNameWithSearch(String host) {
-      InetAddress addr = null;
-      if (host.endsWith(".")) { // already qualified?
-        addr = getByExactName(host); 
-      } else {
-        for (String domain : searchDomains) {
-          String dot = !domain.startsWith(".") ? "." : "";
-          addr = getByExactName(host + dot + domain);
-          if (addr != null) break;
-        }
-      }
-      return addr;
-    }
-
-    // implemented as a separate method to facilitate unit testing
-    InetAddress getInetAddressByName(String host) throws UnknownHostException {
-      return InetAddress.getByName(host);
-    }
-
-    void setSearchDomains(String ... domains) {
-      searchDomains = Arrays.asList(domains);
-    }
-  }
-  
-  /**
-   * This is for testing only!
-   */
-  @VisibleForTesting
-  static void setHostResolver(HostResolver newResolver) {
-    hostResolver = newResolver;
-  }
   
   /**
    * Resolve the uri's hostname and add the default port if not in the uri
@@ -447,7 +276,7 @@ public class NetUtils {
     String fqHost = canonicalizedHostCache.get(host);
     if (fqHost == null) {
       try {
-        fqHost = hostResolver.getByName(host).getHostName();
+        fqHost = SecurityUtil.getByName(host).getHostName();
         // slight race condition, but won't hurt 
         canonicalizedHostCache.put(host, fqHost);
       } catch (UnknownHostException e) {

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java?rev=1230249&r1=1230248&r2=1230249&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java Wed Jan 11 21:02:04 2012
@@ -23,6 +23,8 @@ import java.net.URI;
 import java.net.URL;
 import java.net.UnknownHostException;
 import java.security.AccessController;
+import java.util.Arrays;
+import java.util.List;
 import java.util.ServiceLoader;
 import java.util.Set;
 
@@ -41,6 +43,11 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
 
+import com.google.common.annotations.VisibleForTesting;
+
+//this will need to be replaced someday when there is a suitable replacement
+import sun.net.dns.ResolverConfiguration;
+import sun.net.util.IPAddressUtil;
 import sun.security.jgss.krb5.Krb5Util;
 import sun.security.krb5.Credentials;
 import sun.security.krb5.PrincipalName;
@@ -53,7 +60,10 @@ public class SecurityUtil {
 
   // controls whether buildTokenService will use an ip or host/ip as given
   // by the user
-  private static boolean useIpForTokenService;
+  @VisibleForTesting
+  static boolean useIpForTokenService;
+  @VisibleForTesting
+  static HostResolver hostResolver;
   
   static {
     boolean useIp = new Configuration().getBoolean(
@@ -68,16 +78,9 @@ public class SecurityUtil {
   @InterfaceAudience.Private
   static void setTokenServiceUseIp(boolean flag) {
     useIpForTokenService = flag;
-    NetUtils.setUseQualifiedHostResolver(!flag);
-  }
-  
-  /**
-   * Intended only for temporary use by NetUtils.  Do not use.
-   * @return whether tokens use an IP address
-   */
-  @InterfaceAudience.Private
-  public static boolean getTokenServiceUseIp() {
-    return useIpForTokenService;
+    hostResolver = !useIpForTokenService
+        ? new QualifiedHostResolver()
+        : new StandardHostResolver();
   }
   
   /**
@@ -142,7 +145,7 @@ public class SecurityUtil {
    * it will be removed when the Java behavior is changed.
    * 
    * @param remoteHost Target URL the krb-https client will access
-   * @throws IOException
+   * @throws IOException if the service ticket cannot be retrieved
    */
   public static void fetchServiceTicket(URL remoteHost) throws IOException {
     if(!UserGroupInformation.isSecurityEnabled())
@@ -179,7 +182,7 @@ public class SecurityUtil {
    * @param hostname
    *          the fully-qualified domain name used for substitution
    * @return converted Kerberos principal name
-   * @throws IOException
+   * @throws IOException if the client address cannot be determined
    */
   public static String getServerPrincipal(String principalConfig,
       String hostname) throws IOException {
@@ -204,7 +207,7 @@ public class SecurityUtil {
    * @param addr
    *          InetAddress of the host used for substitution
    * @return converted Kerberos principal name
-   * @throws IOException
+   * @throws IOException if the client address cannot be determined
    */
   public static String getServerPrincipal(String principalConfig,
       InetAddress addr) throws IOException {
@@ -251,7 +254,7 @@ public class SecurityUtil {
    *          the key to look for keytab file in conf
    * @param userNameKey
    *          the key to look for user's Kerberos principal name in conf
-   * @throws IOException
+   * @throws IOException if login fails
    */
   public static void login(final Configuration conf,
       final String keytabFileKey, final String userNameKey) throws IOException {
@@ -271,7 +274,7 @@ public class SecurityUtil {
    *          the key to look for user's Kerberos principal name in conf
    * @param hostname
    *          hostname to use for substitution
-   * @throws IOException
+   * @throws IOException if the config doesn't specify a keytab
    */
   public static void login(final Configuration conf,
       final String keytabFileKey, final String userNameKey, String hostname)
@@ -363,7 +366,7 @@ public class SecurityUtil {
    * Look up the TokenInfo for a given protocol. It searches all known
    * SecurityInfo providers.
    * @param protocol The protocol class to get the information for.
-   * @conf conf Configuration object
+   * @param conf Configuration object
    * @return the TokenInfo or null if it has no KerberosInfo defined
    */
   public static TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
@@ -442,4 +445,155 @@ public class SecurityUtil {
   public static Text buildTokenService(URI uri) {
     return buildTokenService(NetUtils.createSocketAddr(uri.getAuthority()));
   }
+  
+  /**
+   * Resolves a host subject to the security requirements determined by
+   * hadoop.security.token.service.use_ip.
+   * 
+   * @param hostname host or ip to resolve
+   * @return a resolved host
+   * @throws UnknownHostException if the host doesn't exist
+   */
+  @InterfaceAudience.Private
+  public static
+  InetAddress getByName(String hostname) throws UnknownHostException {
+    return hostResolver.getByName(hostname);
+  }
+  
+  interface HostResolver {
+    InetAddress getByName(String host) throws UnknownHostException;    
+  }
+  
+  /**
+   * Uses standard java host resolution
+   */
+  static class StandardHostResolver implements HostResolver {
+    public InetAddress getByName(String host) throws UnknownHostException {
+      return InetAddress.getByName(host);
+    }
+  }
+  
+  /**
+   * This an alternate resolver with important properties that the standard
+   * java resolver lacks:
+   * 1) The hostname is fully qualified.  This avoids security issues if not
+   *    all hosts in the cluster do not share the same search domains.  It
+   *    also prevents other hosts from performing unnecessary dns searches.
+   *    In contrast, InetAddress simply returns the host as given.
+   * 2) The InetAddress is instantiated with an exact host and IP to prevent
+   *    further unnecessary lookups.  InetAddress may perform an unnecessary
+   *    reverse lookup for an IP.
+   * 3) A call to getHostName() will always return the qualified hostname, or
+   *    more importantly, the IP if instantiated with an IP.  This avoids
+   *    unnecessary dns timeouts if the host is not resolvable.
+   * 4) Point 3 also ensures that if the host is re-resolved, ex. during a
+   *    connection re-attempt, that a reverse lookup to host and forward
+   *    lookup to IP is not performed since the reverse/forward mappings may
+   *    not always return the same IP.  If the client initiated a connection
+   *    with an IP, then that IP is all that should ever be contacted.
+   *    
+   * NOTE: this resolver is only used if:
+   *       hadoop.security.token.service.use_ip=false 
+   */
+  protected static class QualifiedHostResolver implements HostResolver {
+    @SuppressWarnings("unchecked")
+    private List<String> searchDomains =
+        ResolverConfiguration.open().searchlist();
+    
+    /**
+     * Create an InetAddress with a fully qualified hostname of the given
+     * hostname.  InetAddress does not qualify an incomplete hostname that
+     * is resolved via the domain search list.
+     * {@link InetAddress#getCanonicalHostName()} will fully qualify the
+     * hostname, but it always return the A record whereas the given hostname
+     * may be a CNAME.
+     * 
+     * @param host a hostname or ip address
+     * @return InetAddress with the fully qualified hostname or ip
+     * @throws UnknownHostException if host does not exist
+     */
+    public InetAddress getByName(String host) throws UnknownHostException {
+      InetAddress addr = null;
+
+      if (IPAddressUtil.isIPv4LiteralAddress(host)) {
+        // use ipv4 address as-is
+        byte[] ip = IPAddressUtil.textToNumericFormatV4(host);
+        addr = InetAddress.getByAddress(host, ip);
+      } else if (IPAddressUtil.isIPv6LiteralAddress(host)) {
+        // use ipv6 address as-is
+        byte[] ip = IPAddressUtil.textToNumericFormatV6(host);
+        addr = InetAddress.getByAddress(host, ip);
+      } else if (host.endsWith(".")) {
+        // a rooted host ends with a dot, ex. "host."
+        // rooted hosts never use the search path, so only try an exact lookup
+        addr = getByExactName(host);
+      } else if (host.contains(".")) {
+        // the host contains a dot (domain), ex. "host.domain"
+        // try an exact host lookup, then fallback to search list
+        addr = getByExactName(host);
+        if (addr == null) {
+          addr = getByNameWithSearch(host);
+        }
+      } else {
+        // it's a simple host with no dots, ex. "host"
+        // try the search list, then fallback to exact host
+        InetAddress loopback = InetAddress.getByName(null);
+        if (host.equalsIgnoreCase(loopback.getHostName())) {
+          addr = InetAddress.getByAddress(host, loopback.getAddress());
+        } else {
+          addr = getByNameWithSearch(host);
+          if (addr == null) {
+            addr = getByExactName(host);
+          }
+        }
+      }
+      // unresolvable!
+      if (addr == null) {
+        throw new UnknownHostException(host);
+      }
+      return addr;
+    }
+
+    InetAddress getByExactName(String host) {
+      InetAddress addr = null;
+      // InetAddress will use the search list unless the host is rooted
+      // with a trailing dot.  The trailing dot will disable any use of the
+      // search path in a lower level resolver.  See RFC 1535.
+      String fqHost = host;
+      if (!fqHost.endsWith(".")) fqHost += ".";
+      try {
+        addr = getInetAddressByName(fqHost);
+        // can't leave the hostname as rooted or other parts of the system
+        // malfunction, ex. kerberos principals are lacking proper host
+        // equivalence for rooted/non-rooted hostnames
+        addr = InetAddress.getByAddress(host, addr.getAddress());
+      } catch (UnknownHostException e) {
+        // ignore, caller will throw if necessary
+      }
+      return addr;
+    }
+
+    InetAddress getByNameWithSearch(String host) {
+      InetAddress addr = null;
+      if (host.endsWith(".")) { // already qualified?
+        addr = getByExactName(host); 
+      } else {
+        for (String domain : searchDomains) {
+          String dot = !domain.startsWith(".") ? "." : "";
+          addr = getByExactName(host + dot + domain);
+          if (addr != null) break;
+        }
+      }
+      return addr;
+    }
+
+    // implemented as a separate method to facilitate unit testing
+    InetAddress getInetAddressByName(String host) throws UnknownHostException {
+      return InetAddress.getByName(host);
+    }
+
+    void setSearchDomains(String ... domains) {
+      searchDomains = Arrays.asList(domains);
+    }
+  }  
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCanonicalization.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCanonicalization.java?rev=1230249&r1=1230248&r2=1230249&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCanonicalization.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCanonicalization.java Wed Jan 11 21:02:04 2012
@@ -25,7 +25,7 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.net.NetUtilsTestResolver;
+import org.apache.hadoop.security.NetUtilsTestResolver;
 import org.apache.hadoop.util.Progressable;
 import org.junit.Test;
 

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java?rev=1230249&r1=1230248&r2=1230249&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java Wed Jan 11 21:02:04 2012
@@ -37,6 +37,7 @@ import org.apache.commons.lang.StringUti
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.NetUtilsTestResolver;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;

Copied: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/NetUtilsTestResolver.java (from r1229743, hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/NetUtilsTestResolver.java)
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/NetUtilsTestResolver.java?p2=hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/NetUtilsTestResolver.java&p1=hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/NetUtilsTestResolver.java&r1=1229743&r2=1230249&rev=1230249&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/NetUtilsTestResolver.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/NetUtilsTestResolver.java Wed Jan 11 21:02:04 2012
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.net;
+package org.apache.hadoop.security;
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -25,7 +25,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.net.NetUtils.QualifiedHostResolver;
+import org.apache.hadoop.security.SecurityUtil.QualifiedHostResolver;
 
 /**
  * provides a dummy dns search resolver with a configurable search path
@@ -41,7 +41,7 @@ public class NetUtilsTestResolver extend
     resolver.addResolvedHost("host.a.b.", "1.1.1.1");
     resolver.addResolvedHost("b-host.b.", "2.2.2.2");
     resolver.addResolvedHost("simple.", "3.3.3.3");    
-    NetUtils.setHostResolver(resolver);
+    SecurityUtil.hostResolver = resolver;
     return resolver;
   }
 
@@ -56,7 +56,8 @@ public class NetUtilsTestResolver extend
     resolvedHosts.put(host, addr);
   }
 
-  InetAddress getInetAddressByName(String host) throws UnknownHostException {
+  @Override
+  public InetAddress getInetAddressByName(String host) throws UnknownHostException {
     hostSearches.add(host);
     if (!resolvedHosts.containsKey(host)) {
       throw new UnknownHostException(host);
@@ -64,11 +65,21 @@ public class NetUtilsTestResolver extend
     return resolvedHosts.get(host);
   }
 
-  String[] getHostSearches() {
+  @Override
+  public InetAddress getByExactName(String host) {
+    return super.getByExactName(host);
+  }
+  
+  @Override
+  public InetAddress getByNameWithSearch(String host) {
+    return super.getByNameWithSearch(host);
+  }
+  
+  public String[] getHostSearches() {
     return hostSearches.toArray(new String[0]);
   }
 
-  void reset() {
+  public void reset() {
     hostSearches.clear();
   }
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java?rev=1230249&r1=1230248&r2=1230249&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java Wed Jan 11 21:02:04 2012
@@ -225,7 +225,7 @@ public class TestSecurityUtil {
     assertTrue(!addr.isUnresolved());
     // don't know what the standard resolver will return for hostname.
     // should be host for host; host or ip for ip is ambiguous
-    if (!SecurityUtil.getTokenServiceUseIp()) {
+    if (!SecurityUtil.useIpForTokenService) {
       assertEquals(host, addr.getHostName());
       assertEquals(host, addr.getAddress().getHostName());
     }