[ https://issues.apache.org/jira/browse/SPARK-22778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16289822#comment-16289822 ]
Yinan Li edited comment on SPARK-22778 at 12/13/17 8:24 PM: ------------------------------------------------------------ Just some background on this. The validation and parsing of k8s master url has been moved to SparkSubmit as being suggested in the review. The parsed master URL (https://... for example) is appended a {{k8s}} prefix after the parsing to satisfy {{KubernetesClusterManager}}, whose {{canCreate}} method is based on if the master URL starts {{k8s}}. That's why you see the {{k8s:}} prefix. The issue seems that in the driver pod {{SparkContext}} could not find {{KubernetesClusterManager}} based on the debug messages I added. The code that triggered the error (with the debugging I added) is as follows: {code:java} private def getClusterManager(url: String): Option[ExternalClusterManager] = { val loader = Utils.getContextOrSparkClassLoader val serviceLoaders = ServiceLoader.load(classOf[ExternalClusterManager], loader).asScala serviceLoaders.foreach { loader => logInfo(s"Found the following external cluster manager: $loader") } val filteredServiceLoaders = serviceLoaders.filter(_.canCreate(url)) if (filteredServiceLoaders.size > 1) { throw new SparkException( s"Multiple external cluster managers registered for the url $url: $serviceLoaders") } else if (filteredServiceLoaders.isEmpty) { logWarning(s"No external cluster manager registered for url $url") } filteredServiceLoaders.headOption } {code} And I got the following: {code:java} No external cluster manager registered for url k8s:https://35.226.8.173 {code} was (Author: liyinan926): Just some background on this. The validation and parsing of k8s master url has been moved to SparkSubmit as being suggested in the review. The parsed master URL (https://... for example) is appended a {{k8s}} prefix after the parsing to satisfy {{KubernetesClusterManager}}, whose {{canCreate}} method is based on if the master URL starts {{k8s}}. That's why you see the {{k8s:}} prefix. The issue seems that in the driver pod {{SparkContext}} could not find {{KubernetesClusterManager}} based on the debug messages I added: {code:scala} private def getClusterManager(url: String): Option[ExternalClusterManager] = { val loader = Utils.getContextOrSparkClassLoader val serviceLoaders = ServiceLoader.load(classOf[ExternalClusterManager], loader).asScala serviceLoaders.foreach { loader => logInfo(s"Found the following external cluster manager: $loader") } val filteredServiceLoaders = serviceLoaders.filter(_.canCreate(url)) if (filteredServiceLoaders.size > 1) { throw new SparkException( s"Multiple external cluster managers registered for the url $url: $serviceLoaders") } else if (filteredServiceLoaders.isEmpty) { logWarning(s"No external cluster manager registered for url $url") } filteredServiceLoaders.headOption } {code} And I got the following: {code:java} No external cluster manager registered for url k8s:https://35.226.8.173 {code} > Kubernetes scheduler at master failing to run applications successfully > ----------------------------------------------------------------------- > > Key: SPARK-22778 > URL: https://issues.apache.org/jira/browse/SPARK-22778 > Project: Spark > Issue Type: Bug > Components: Kubernetes > Affects Versions: 2.3.0 > Reporter: Anirudh Ramanathan > > Building images based on master and deploying Spark PI results in the > following error. > 2017-12-13 19:57:19 INFO SparkContext:54 - Successfully stopped SparkContext > Exception in thread "main" org.apache.spark.SparkException: Could not parse > Master URL: 'k8s:https://xx.yy.zz.ww' > at > org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$createTaskScheduler(SparkContext.scala:2741) > at org.apache.spark.SparkContext.<init>(SparkContext.scala:496) > at org.apache.spark.SparkContext$.getOrCreate(SparkContext.scala:2490) > at > org.apache.spark.sql.SparkSession$Builder$$anonfun$7.apply(SparkSession.scala:927) > at > org.apache.spark.sql.SparkSession$Builder$$anonfun$7.apply(SparkSession.scala:918) > at scala.Option.getOrElse(Option.scala:121) > at > org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:918) > at org.apache.spark.examples.SparkPi$.main(SparkPi.scala:31) > at org.apache.spark.examples.SparkPi.main(SparkPi.scala) > 2017-12-13 19:57:19 INFO ShutdownHookManager:54 - Shutdown hook called > 2017-12-13 19:57:19 INFO ShutdownHookManager:54 - Deleting directory > /tmp/spark-b47515c2-6750-4a37-aa68-6ee12da5d2bd > This is likely an artifact seen because of changes in master, or our > submission code in the reviews. We haven't seen this on our fork. Hopefully > once integration tests are ported against upstream/master, we will catch > these issues earlier. -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org