[jira] [Comment Edited] (SPARK-22778) Kubernetes scheduler at master failing to run applications successfully
[ https://issues.apache.org/jira/browse/SPARK-22778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16289868#comment-16289868 ] Anirudh Ramanathan edited comment on SPARK-22778 at 12/13/17 8:41 PM: -- I've verified that the image contains the right jars. One more thing that changed from underneath us is https://github.com/apache/spark/pull/19631. Not sure yet if that's related. was (Author: foxish): I've verified that the image contains the right jars. One more thing that changed from underneath us is https://github.com/apache/spark/pull/19631. Not sure if that's related. > 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 >Priority: Critical > > 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.(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
[jira] [Comment Edited] (SPARK-22778) Kubernetes scheduler at master failing to run applications successfully
[ https://issues.apache.org/jira/browse/SPARK-22778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16289836#comment-16289836 ] Matt Cheah edited comment on SPARK-22778 at 12/13/17 8:28 PM: -- The {{canCreate}} method for {{KubernetesClusterManager}} should match that URI. The primary possibility I can think of is that the {{KubernetesClusterManager}} isn't being service loaded at all, which would imply that {{spark-kubernetes}} isn't on the classpath. Can we verify that the Docker image contains all of the correct jars? was (Author: mcheah): The `canCreate` method for `KubernetesClusterManager` should match that URI. The primary possibility I can think of is that the KubernetesClusterManager isn't being service loaded at all, which would imply that `spark-kubernetes` isn't on the classpath. Can we verify that the Docker image contains all of the correct jars? > 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.(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
[jira] [Comment Edited] (SPARK-22778) Kubernetes scheduler at master failing to run applications successfully
[ https://issues.apache.org/jira/browse/SPARK-22778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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.(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.
[jira] [Comment Edited] (SPARK-22778) Kubernetes scheduler at master failing to run applications successfully
[ https://issues.apache.org/jira/browse/SPARK-22778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16289818#comment-16289818 ] Anirudh Ramanathan edited comment on SPARK-22778 at 12/13/17 8:23 PM: -- I submitted it as `k8s://https://xx.yy.zz.ww` to spark submit. However, it seems there is some change in how the validation of said URL occurs on the client-side - which makes us strip out the k8s and add it back in the above format. That might be at fault here. Here's my full spark-submit command: bin/spark-submit \ --deploy-mode cluster \ --class org.apache.spark.examples.SparkPi \ --master k8s://https://xx.yy.zz.ww \ --conf spark.executor.instances=5 \ --conf spark.app.name=spark-pi \ --conf spark.kubernetes.driver.docker.image=foxish/spark-driver:spark-k8s-master-13dec-11-56 \ --conf spark.kubernetes.executor.docker.image=foxish/spark-executor:spark-k8s-master-13dec-11-56 \ local:///opt/spark/examples/jars/spark-examples_2.11-2.3.0-SNAPSHOT.jar was (Author: foxish): I submitted it as `k8s://https://xx.yy.zz.ww` to spark submit. However, it seems there is some change in how the validation of said URL occurs on the client-side - which makes us strip out the k8s and add it back in the above format. That might be at fault here. > 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.(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