浏览代码

YARN-9646. DistributedShell tests failed to bind to a local host name. (Contributed by Ray Yang)

Haibo Chen 5 年之前
父节点
当前提交
5915c902aa

+ 2 - 64
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -33,7 +33,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintWriter;
 import java.io.UncheckedIOException;
-import java.net.InetAddress;
 import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
@@ -53,7 +52,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -401,7 +399,6 @@ public class TestDistributedShell {
     YarnClient yarnClient = YarnClient.createYarnClient();
     yarnClient.init(new Configuration(yarnCluster.getConfig()));
     yarnClient.start();
-    String hostName = NetUtils.getHostname();
 
     boolean verified = false;
     String errorMessage = "";
@@ -420,10 +417,9 @@ public class TestDistributedShell {
         continue;
       }
       errorMessage =
-          "Expected host name to start with '" + hostName + "', was '"
-              + appReport.getHost() + "'. Expected rpc port to be '-1', was '"
+          "'. Expected rpc port to be '-1', was '"
               + appReport.getRpcPort() + "'.";
-      if (checkHostname(appReport.getHost()) && appReport.getRpcPort() == -1) {
+      if (appReport.getRpcPort() == -1) {
         verified = true;
       }
 
@@ -730,64 +726,6 @@ public class TestDistributedShell {
     return argsList.toArray(new String[argsList.size()]);
   }
 
-  /*
-   * NetUtils.getHostname() returns a string in the form "hostname/ip".
-   * Sometimes the hostname we get is the FQDN and sometimes the short name. In
-   * addition, on machines with multiple network interfaces, it runs any one of
-   * the ips. The function below compares the returns values for
-   * NetUtils.getHostname() accounting for the conditions mentioned.
-   */
-  private boolean checkHostname(String appHostname) throws Exception {
-
-    String hostname = NetUtils.getHostname();
-    if (hostname.equals(appHostname)) {
-      return true;
-    }
-
-    Assert.assertTrue("Unknown format for hostname " + appHostname,
-      appHostname.contains("/"));
-    Assert.assertTrue("Unknown format for hostname " + hostname,
-      hostname.contains("/"));
-
-    String[] appHostnameParts = appHostname.split("/");
-    String[] hostnameParts = hostname.split("/");
-
-    return (compareFQDNs(appHostnameParts[0], hostnameParts[0]) && checkIPs(
-      hostnameParts[0], hostnameParts[1], appHostnameParts[1]));
-  }
-
-  private boolean compareFQDNs(String appHostname, String hostname)
-      throws Exception {
-    if (appHostname.equals(hostname)) {
-      return true;
-    }
-    String appFQDN = InetAddress.getByName(appHostname).getCanonicalHostName();
-    String localFQDN = InetAddress.getByName(hostname).getCanonicalHostName();
-    return appFQDN.equals(localFQDN);
-  }
-
-  private boolean checkIPs(String hostname, String localIP, String appIP)
-      throws Exception {
-
-    if (localIP.equals(appIP)) {
-      return true;
-    }
-    boolean appIPCheck = false;
-    boolean localIPCheck = false;
-    InetAddress[] addresses = InetAddress.getAllByName(hostname);
-    for (InetAddress ia : addresses) {
-      if (ia.getHostAddress().equals(appIP)) {
-        appIPCheck = true;
-        continue;
-      }
-      if (ia.getHostAddress().equals(localIP)) {
-        localIPCheck = true;
-      }
-    }
-    return (appIPCheck && localIPCheck);
-
-  }
-
   protected String getSleepCommand(int sec) {
     // Windows doesn't have a sleep command, ping -n does the trick
     return Shell.WINDOWS ? "ping -n " + (sec + 1) + " 127.0.0.1 >nul"

+ 1 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

@@ -20,9 +20,6 @@ package org.apache.hadoop.yarn.server;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
@@ -39,7 +36,6 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.net.ServerSocketUtil;
@@ -470,21 +466,7 @@ public class MiniYARNCluster extends CompositeService {
   }
 
   public static String getHostname() {
-    try {
-      String hostname = InetAddress.getLocalHost().getHostName();
-      // Create InetSocketAddress to see whether it is resolved or not.
-      // If not, just return "localhost".
-      InetSocketAddress addr =
-          NetUtils.createSocketAddrForHost(hostname, 1);
-      if (addr.isUnresolved()) {
-        return "localhost";
-      } else {
-        return hostname;
-      }
-    }
-    catch (UnknownHostException ex) {
-      throw new RuntimeException(ex);
-    }
+    return "localhost";
   }
 
   private class ResourceManagerWrapper extends AbstractService {