[ https://issues.apache.org/jira/browse/SPARK-27937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16859001#comment-16859001 ]
Dhruve Ashar edited comment on SPARK-27937 at 6/7/19 9:27 PM: -------------------------------------------------------------- The exception that we started encountering is while spark tries to create a path of the logic nameservice or nameservice id configured as a part of HDFS federation as a part of the code here: https://github.com/apache/spark/blob/v2.4.3/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala#L215 {code:java} 19/05/20 08:48:42 INFO SecurityManager: Changing modify acls groups to: 19/05/20 08:48:42 INFO SecurityManager: SecurityManager: authentication enabled; ui acls enabled; users with view permissions: Set(...); groups with view permissions: Set(....); users with modify permissions: Set(....); groups with modify permissions: Set(.....) 19/05/20 08:48:43 INFO Client: Deleted staging directory hdfs://..........:8020/user/abc/.sparkStaging/application_123456_123456 Exception in thread "main" java.io.IOException: Cannot create proxy with unresolved address: abcabcabc-nn1:8020 at org.apache.hadoop.hdfs.NameNodeProxiesClient.createNonHAProxyWithClientProtocol(NameNodeProxiesClient.java:345) at org.apache.hadoop.hdfs.NameNodeProxiesClient.createProxyWithClientProtocol(NameNodeProxiesClient.java:133) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:351) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:285) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:160) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2821) at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:100) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2892) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2874) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389) at org.apache.hadoop.fs.Path.getFileSystem(Path.java:356) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$$anonfun$5$$anonfun$apply$2.apply(YarnSparkHadoopUtil.scala:215) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$$anonfun$5$$anonfun$apply$2.apply(YarnSparkHadoopUtil.scala:214) at scala.Option.map(Option.scala:146) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$$anonfun$5.apply(YarnSparkHadoopUtil.scala:214) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$$anonfun$5.apply(YarnSparkHadoopUtil.scala:213) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241) at scala.collection.mutable.ArrayOps$ofRef.flatMap(ArrayOps.scala:186) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$.hadoopFSsToAccess(YarnSparkHadoopUtil.scala:213) at org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager$$anonfun$1.apply(YARNHadoopDelegationTokenManager.scala:43) at org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager$$anonfun$1.apply(YARNHadoopDelegationTokenManager.scala:43) at org.apache.spark.deploy.security.HadoopFSDelegationTokenProvider.obtainDelegationTokens(HadoopFSDelegationTokenProvider.scala:48) {code} was (Author: dhruve ashar): The exception that we started encountering is while spark tries to create a path of the logic nameservice or nameservice id configured as a part of HDFS federation. {code:java} 19/05/20 08:48:42 INFO SecurityManager: Changing modify acls groups to: 19/05/20 08:48:42 INFO SecurityManager: SecurityManager: authentication enabled; ui acls enabled; users with view permissions: Set(...); groups with view permissions: Set(....); users with modify permissions: Set(....); groups with modify permissions: Set(.....) 19/05/20 08:48:43 INFO Client: Deleted staging directory hdfs://..........:8020/user/abc/.sparkStaging/application_123456_123456 Exception in thread "main" java.io.IOException: Cannot create proxy with unresolved address: abcabcabc-nn1:8020 at org.apache.hadoop.hdfs.NameNodeProxiesClient.createNonHAProxyWithClientProtocol(NameNodeProxiesClient.java:345) at org.apache.hadoop.hdfs.NameNodeProxiesClient.createProxyWithClientProtocol(NameNodeProxiesClient.java:133) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:351) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:285) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:160) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2821) at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:100) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2892) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2874) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389) at org.apache.hadoop.fs.Path.getFileSystem(Path.java:356) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$$anonfun$5$$anonfun$apply$2.apply(YarnSparkHadoopUtil.scala:215) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$$anonfun$5$$anonfun$apply$2.apply(YarnSparkHadoopUtil.scala:214) at scala.Option.map(Option.scala:146) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$$anonfun$5.apply(YarnSparkHadoopUtil.scala:214) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$$anonfun$5.apply(YarnSparkHadoopUtil.scala:213) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241) at scala.collection.mutable.ArrayOps$ofRef.flatMap(ArrayOps.scala:186) at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$.hadoopFSsToAccess(YarnSparkHadoopUtil.scala:213) at org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager$$anonfun$1.apply(YARNHadoopDelegationTokenManager.scala:43) at org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager$$anonfun$1.apply(YARNHadoopDelegationTokenManager.scala:43) at org.apache.spark.deploy.security.HadoopFSDelegationTokenProvider.obtainDelegationTokens(HadoopFSDelegationTokenProvider.scala:48) {code} > Revert changes introduced as a part of Automatic namespace discovery > [SPARK-24149] > ---------------------------------------------------------------------------------- > > Key: SPARK-27937 > URL: https://issues.apache.org/jira/browse/SPARK-27937 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 2.4.3 > Reporter: Dhruve Ashar > Priority: Major > > Spark fails to launch for a valid deployment of HDFS while trying to get > tokens for a logical nameservice instead of an actual namenode (with HDFS > federation enabled). > On inspecting the source code closely, it is unclear why we were doing it and > based on the context from SPARK-24149, it solves a very specific use case of > getting the tokens for only those namenodes which are configured for HDFS > federation in the same cluster. IMHO these are better left to the user to > specify explicitly. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org