[
https://issues.apache.org/jira/browse/FLINK-9684?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Chesnay Schepler updated FLINK-9684:
------------------------------------
Component/s: History Server
> HistoryServerArchiveFetcher not working properly with secure hdfs cluster
> -------------------------------------------------------------------------
>
> Key: FLINK-9684
> URL: https://issues.apache.org/jira/browse/FLINK-9684
> Project: Flink
> Issue Type: Bug
> Components: History Server
> Affects Versions: 1.4.2
> Reporter: Ethan Li
> Assignee: Ethan Li
> Priority: Major
> Labels: pull-request-available
> Fix For: 1.6.0, 1.5.1
>
>
> With my current setup, jobmanager and taskmanager are able to talk to hdfs
> cluster (with kerberos setup). However, running history server gets:
>
>
> {code:java}
> 2018-06-27 19:03:32,080 WARN org.apache.hadoop.ipc.Client - Exception
> encountered while connecting to the server :
> java.lang.IllegalArgumentException: Failed to specify server's Kerberos
> principal name
> 2018-06-27 19:03:32,085 ERROR
> org.apache.flink.runtime.webmonitor.history.HistoryServerArchiveFetcher -
> Failed to access job archive location for path
> hdfs://openqe11blue-n2.blue.ygrid.yahoo.com/tmp/flink/openstorm10-blue/jmarchive.
> java.io.IOException: Failed on local exception: java.io.IOException:
> java.lang.IllegalArgumentException: Failed to specify server's Kerberos
> principal name; Host Details : local host is:
> "openstorm10blue-n2.blue.ygrid.yahoo.com/10.215.79.35"; destination host is:
> "openqe11blue-n2.blue.ygri
> d.yahoo.com":8020;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:764)
> at org.apache.hadoop.ipc.Client.call(Client.java:1414)
> at org.apache.hadoop.ipc.Client.call(Client.java:1363)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
> at com.sun.proxy.$Proxy9.getListing(Unknown Source)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:190)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:103)
> at com.sun.proxy.$Proxy9.getListing(Unknown Source)
> at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getListing(ClientNamenodeProtocolTranslatorPB.java:515)
> at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1743)
> at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1726)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem.listStatusInternal(DistributedFileSystem.java:650)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem.access$600(DistributedFileSystem.java:102)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:712)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:708)
> at
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem.listStatus(DistributedFileSystem.java:708)
> at
> org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.listStatus(HadoopFileSystem.java:146)
> at
> org.apache.flink.runtime.webmonitor.history.HistoryServerArchiveFetcher$JobArchiveFetcherTask.run(HistoryServerArchiveFetcher.java:139)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.io.IOException: java.lang.IllegalArgumentException: Failed to
> specify server's Kerberos principal name
> at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:677)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1556)
> at
> org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:640)
> at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:724)
> at org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:367)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1462)
> at org.apache.hadoop.ipc.Client.call(Client.java:1381)
> ... 28 more
> {code}
>
>
> Changed LOG Level to DEBUG and seeing
>
> {code:java}
> 2018-06-27 19:03:30,931 INFO
> org.apache.flink.runtime.webmonitor.history.HistoryServer - Enabling SSL for
> the history server.
> 2018-06-27 19:03:30,931 DEBUG org.apache.flink.runtime.net.SSLUtils -
> Creating server SSL context from configuration
> 2018-06-27 19:03:31,091 DEBUG org.apache.flink.core.fs.FileSystem - Loading
> extension file systems via services
> 2018-06-27 19:03:31,094 DEBUG org.apache.flink.core.fs.FileSystem - Added
> file system maprfs:org.apache.flink.runtime.fs.maprfs.MapRFsFactory
> 2018-06-27 19:03:31,102 DEBUG org.apache.flink.runtime.util.HadoopUtils -
> Cannot find hdfs-default configuration-file path in Flink config.
> 2018-06-27 19:03:31,102 DEBUG org.apache.flink.runtime.util.HadoopUtils -
> Cannot find hdfs-site configuration-file path in Flink config.
> 2018-06-27 19:03:31,102 DEBUG org.apache.flink.runtime.util.HadoopUtils -
> Could not find Hadoop configuration via any of the supported methods (Flink
> configuration, environment variables).
> 2018-06-27 19:03:31,178 DEBUG
> org.apache.flink.runtime.fs.hdfs.HadoopFsFactory - Instantiating for file
> system scheme hdfs Hadoop File System
> org.apache.hadoop.hdfs.DistributedFileSystem
> 2018-06-27 19:03:31,829 INFO
> org.apache.flink.runtime.webmonitor.history.HistoryServerArchiveFetcher -
> Monitoring directory
> hdfs://openqe11blue-n2.blue.ygrid.yahoo.com/tmp/flink/openstorm10-blue/jmarchive
> for archived jobs.
> {code}
>
>
> The root cause is
> https://github.com/apache/flink/blob/release-1.4.2/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java#L169
>
> {code:java}
> FileSystem refreshFS = refreshPath.getFileSystem();
> {code}
>
> The getFileSystem() is being called before
> {code:java}
> FileSystem.initialize(xxx){code}
> ever happened.
> So it will call
> [https://github.com/apache/flink/blob/release-1.4.2/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java#L388-L390]
>
> {code:java}
> if (FS_FACTORIES.isEmpty()) {
> initialize(new Configuration());
> }
> {code}
> and because the configuration is empty, it won't be able to connect to hdfs
> correctly.
>
> A workaround is to set HADOOP_CONF_DIR or HADOOP_HOME environment variables.
> But we should fix this since we have
> {code:java}
> fs.hdfs.hadoopconf
> {code}
> config, otherwise it will be confusing to users.
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)