You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "He Xiaoqiao (JIRA)" <ji...@apache.org> on 2018/10/19 07:47:00 UTC

[jira] [Created] (HADOOP-15864) Job submitter / executor fail when SBN domain name can not resolved

He Xiaoqiao created HADOOP-15864:
------------------------------------

             Summary: Job submitter / executor fail when SBN domain name can not resolved
                 Key: HADOOP-15864
                 URL: https://issues.apache.org/jira/browse/HADOOP-15864
             Project: Hadoop Common
          Issue Type: Bug
            Reporter: He Xiaoqiao
            Assignee: He Xiaoqiao


Job submit failure and Task executes failure if Standby NameNode domain name can not resolved on HDFS HA with DelegationToken feature.

This issue is triggered when create {{ConfiguredFailoverProxyProvider}} instance which invoke {{HAUtil.cloneDelegationTokenForLogicalUri}} in HA mode with Security. Since in HDFS HA mode UGI need include separate token for each NameNode in order to dealing with Active-Standby switch, the double tokens' content is same of course. 
However when #setTokenService in {{HAUtil.cloneDelegationTokenForLogicalUri}} it checks whether the address of NameNode has been resolved or not, if Not, throw #IllegalArgumentException upon, then job submitter/ task executor fail.

HDFS-8068 and HADOOP-12125 try to fix it, but I don't think the two tickets resolve completely.
Another questions many guys consider is why NameNode domain name can not resolve? I think there are many scenarios, for instance node replace when meet fault, and refresh DNS sometimes. Anyway, Standby NameNode failure should not impact Hadoop cluster stability in my opinion.

a. code ref: org.apache.hadoop.security.SecurityUtil line373-386
{code:java}
  public static Text buildTokenService(InetSocketAddress addr) {
    String host = null;
    if (useIpForTokenService) {
      if (addr.isUnresolved()) { // host has no ip address
        throw new IllegalArgumentException(
            new UnknownHostException(addr.getHostName())
        );
      }
      host = addr.getAddress().getHostAddress();
    } else {
      host = StringUtils.toLowerCase(addr.getHostName());
    }
    return new Text(host + ":" + addr.getPort());
  }
{code}

b.exception log ref:
{code:xml}
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:227)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.IOException: Couldn't create proxy provider class org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider
at org.apache.hadoop.hdfs.NameNodeProxies.createFailoverProxyProvider(NameNodeProxies.java:515)
at org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:170)
at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:761)
at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:691)
at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:150)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2713)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:93)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2747)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2729)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:385)
at org.apache.hadoop.fs.viewfs.ChRootedFileSystem.<init>(ChRootedFileSystem.java:106)
at org.apache.hadoop.fs.viewfs.ViewFileSystem$1.getTargetFileSystem(ViewFileSystem.java:178)
at org.apache.hadoop.fs.viewfs.ViewFileSystem$1.getTargetFileSystem(ViewFileSystem.java:172)
at org.apache.hadoop.fs.viewfs.InodeTree.createLink(InodeTree.java:303)
at org.apache.hadoop.fs.viewfs.InodeTree.<init>(InodeTree.java:377)
at org.apache.hadoop.fs.viewfs.ViewFileSystem$1.<init>(ViewFileSystem.java:172)
at org.apache.hadoop.fs.viewfs.ViewFileSystem.initialize(ViewFileSystem.java:172)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2713)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:93)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2747)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2729)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:385)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:176)
at org.apache.hadoop.mapred.JobConf.getWorkingDirectory(JobConf.java:665)
... 35 more
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.GeneratedConstructorAccessor14.newInstance(Unknown Source)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
at org.apache.hadoop.hdfs.NameNodeProxies.createFailoverProxyProvider(NameNodeProxies.java:498)
... 58 more
Caused by: java.lang.IllegalArgumentException: java.net.UnknownHostException: standbynamenode
at org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:390)
at org.apache.hadoop.security.SecurityUtil.setTokenService(SecurityUtil.java:369)
at org.apache.hadoop.hdfs.HAUtil.cloneDelegationTokenForLogicalUri(HAUtil.java:317)
at org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider.<init>(ConfiguredFailoverProxyProvider.java:132)
at org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider.<init>(ConfiguredFailoverProxyProvider.java:84)
... 62 more
Caused by: java.net.UnknownHostException: standbynamenode
... 67 more
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org