Re: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN)
For posterity, I found the root cause and filed a JIRA: https://issues.apache.org/jira/browse/SPARK-21960. I plan to open a pull request with the minor fix. From: Karthik Palaniappan Sent: Friday, September 1, 2017 9:49 AM To: Akhil Das Cc: user@spark.apache.org; t...@databricks.com Subject: Re: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN) Any ideas @Tathagata? I'd be happy to contribute a patch if you can point me in the right direction. From: Karthik Palaniappan Sent: Friday, August 25, 2017 9:15 AM To: Akhil Das Cc: user@spark.apache.org; t...@databricks.com Subject: RE: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN) You have to set spark.executor.instances=0 in a streaming application with dynamic allocation: https://github.com/tdas/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala#L207. I originally had it set to a positive value, and explicitly set it to 0 after hitting that error. Setting executor cores > 1 seems like reasonable advice in general, but that shouldn’t be my issue here, right? From: Akhil Das<mailto:ak...@hacked.work> Sent: Thursday, August 24, 2017 2:34 AM To: Karthik Palaniappan<mailto:karthik...@hotmail.com> Cc: user@spark.apache.org<mailto:user@spark.apache.org>; t...@databricks.com<mailto:t...@databricks.com> Subject: Re: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN) Have you tried setting spark.executor.instances=0 to a positive non-zero value? Also, since its a streaming application set executor cores > 1. On Wed, Aug 23, 2017 at 3:38 AM, Karthik Palaniappan mailto:karthik...@hotmail.com>> wrote: I ran the HdfsWordCount example using this command: spark-submit run-example \ --conf spark.streaming.dynamicAllocation.enabled=true \ --conf spark.executor.instances=0 \ --conf spark.dynamicAllocation.enabled=false \ --conf spark.master=yarn \ --conf spark.submit.deployMode=client \ org.apache.spark.examples.streaming.HdfsWordCount /foo I tried it on both Spark 2.1.1 (through HDP 2.6) and Spark 2.2.0 (through Google Dataproc 1.2), and I get the same message repeatedly that Spark cannot allocate any executors. 17/08/22 19:34:57 INFO org.spark_project.jetty.util.log: Logging initialized @1694ms 17/08/22 19:34:57 INFO org.spark_project.jetty.server.Server: jetty-9.3.z-SNAPSHOT 17/08/22 19:34:57 INFO org.spark_project.jetty.server.Server: Started @1756ms 17/08/22 19:34:57 INFO org.spark_project.jetty.server.AbstractConnector: Started ServerConnector@578782d6{HTTP/1.1,[http/1.1]}{0.0.0.0:4040<http://0.0.0.0:4040>} 17/08/22 19:34:58 INFO com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystemBase: GHFS version: 1.6.1-hadoop2 17/08/22 19:34:58 INFO org.apache.hadoop.yarn.client.RMProxy: Connecting to ResourceManager at hadoop-m/10.240.1.92:8032<http://10.240.1.92:8032> 17/08/22 19:35:00 INFO org.apache.hadoop.yarn.client.api.impl.YarnClientImpl: Submitted application application_1503036971561_0022 17/08/22 19:35:04 WARN org.apache.spark.streaming.StreamingContext: Dynamic Allocation is enabled for this application. Enabling Dynamic allocation for Spark Streaming applications can cause data loss if Write Ahead Log is not enabled for non-replayable sources like Flume. See the programming guide for details on how to enable the Write Ahead Log. 17/08/22 19:35:21 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources 17/08/22 19:35:36 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources 17/08/22 19:35:51 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources I confirmed that the YARN cluster has enough memory for dozens of executors, and verified that the application allocates executors when using Core's spark.dynamicAllocation.enabled=true, and leaving spark.streaming.dynamicAllocation.enabled=false. Is streaming dynamic allocation actually supported? Sean Owen suggested it might have been experimental: https://issues.apache.org/jira/browse/SPARK-21792. -- Cheers!
Re: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN)
Any ideas @Tathagata? I'd be happy to contribute a patch if you can point me in the right direction. From: Karthik Palaniappan Sent: Friday, August 25, 2017 9:15 AM To: Akhil Das Cc: user@spark.apache.org; t...@databricks.com Subject: RE: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN) You have to set spark.executor.instances=0 in a streaming application with dynamic allocation: https://github.com/tdas/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala#L207. I originally had it set to a positive value, and explicitly set it to 0 after hitting that error. Setting executor cores > 1 seems like reasonable advice in general, but that shouldn’t be my issue here, right? From: Akhil Das<mailto:ak...@hacked.work> Sent: Thursday, August 24, 2017 2:34 AM To: Karthik Palaniappan<mailto:karthik...@hotmail.com> Cc: user@spark.apache.org<mailto:user@spark.apache.org>; t...@databricks.com<mailto:t...@databricks.com> Subject: Re: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN) Have you tried setting spark.executor.instances=0 to a positive non-zero value? Also, since its a streaming application set executor cores > 1. On Wed, Aug 23, 2017 at 3:38 AM, Karthik Palaniappan mailto:karthik...@hotmail.com>> wrote: I ran the HdfsWordCount example using this command: spark-submit run-example \ --conf spark.streaming.dynamicAllocation.enabled=true \ --conf spark.executor.instances=0 \ --conf spark.dynamicAllocation.enabled=false \ --conf spark.master=yarn \ --conf spark.submit.deployMode=client \ org.apache.spark.examples.streaming.HdfsWordCount /foo I tried it on both Spark 2.1.1 (through HDP 2.6) and Spark 2.2.0 (through Google Dataproc 1.2), and I get the same message repeatedly that Spark cannot allocate any executors. 17/08/22 19:34:57 INFO org.spark_project.jetty.util.log: Logging initialized @1694ms 17/08/22 19:34:57 INFO org.spark_project.jetty.server.Server: jetty-9.3.z-SNAPSHOT 17/08/22 19:34:57 INFO org.spark_project.jetty.server.Server: Started @1756ms 17/08/22 19:34:57 INFO org.spark_project.jetty.server.AbstractConnector: Started ServerConnector@578782d6{HTTP/1.1,[http/1.1]}{0.0.0.0:4040<http://0.0.0.0:4040>} 17/08/22 19:34:58 INFO com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystemBase: GHFS version: 1.6.1-hadoop2 17/08/22 19:34:58 INFO org.apache.hadoop.yarn.client.RMProxy: Connecting to ResourceManager at hadoop-m/10.240.1.92:8032<http://10.240.1.92:8032> 17/08/22 19:35:00 INFO org.apache.hadoop.yarn.client.api.impl.YarnClientImpl: Submitted application application_1503036971561_0022 17/08/22 19:35:04 WARN org.apache.spark.streaming.StreamingContext: Dynamic Allocation is enabled for this application. Enabling Dynamic allocation for Spark Streaming applications can cause data loss if Write Ahead Log is not enabled for non-replayable sources like Flume. See the programming guide for details on how to enable the Write Ahead Log. 17/08/22 19:35:21 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources 17/08/22 19:35:36 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources 17/08/22 19:35:51 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources I confirmed that the YARN cluster has enough memory for dozens of executors, and verified that the application allocates executors when using Core's spark.dynamicAllocation.enabled=true, and leaving spark.streaming.dynamicAllocation.enabled=false. Is streaming dynamic allocation actually supported? Sean Owen suggested it might have been experimental: https://issues.apache.org/jira/browse/SPARK-21792. -- Cheers!
RE: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN)
You have to set spark.executor.instances=0 in a streaming application with dynamic allocation: https://github.com/tdas/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala#L207. I originally had it set to a positive value, and explicitly set it to 0 after hitting that error. Setting executor cores > 1 seems like reasonable advice in general, but that shouldn’t be my issue here, right? From: Akhil Das<mailto:ak...@hacked.work> Sent: Thursday, August 24, 2017 2:34 AM To: Karthik Palaniappan<mailto:karthik...@hotmail.com> Cc: user@spark.apache.org<mailto:user@spark.apache.org>; t...@databricks.com<mailto:t...@databricks.com> Subject: Re: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN) Have you tried setting spark.executor.instances=0 to a positive non-zero value? Also, since its a streaming application set executor cores > 1. On Wed, Aug 23, 2017 at 3:38 AM, Karthik Palaniappan mailto:karthik...@hotmail.com>> wrote: I ran the HdfsWordCount example using this command: spark-submit run-example \ --conf spark.streaming.dynamicAllocation.enabled=true \ --conf spark.executor.instances=0 \ --conf spark.dynamicAllocation.enabled=false \ --conf spark.master=yarn \ --conf spark.submit.deployMode=client \ org.apache.spark.examples.streaming.HdfsWordCount /foo I tried it on both Spark 2.1.1 (through HDP 2.6) and Spark 2.2.0 (through Google Dataproc 1.2), and I get the same message repeatedly that Spark cannot allocate any executors. 17/08/22 19:34:57 INFO org.spark_project.jetty.util.log: Logging initialized @1694ms 17/08/22 19:34:57 INFO org.spark_project.jetty.server.Server: jetty-9.3.z-SNAPSHOT 17/08/22 19:34:57 INFO org.spark_project.jetty.server.Server: Started @1756ms 17/08/22 19:34:57 INFO org.spark_project.jetty.server.AbstractConnector: Started ServerConnector@578782d6{HTTP/1.1,[http/1.1]}{0.0.0.0:4040<http://0.0.0.0:4040>} 17/08/22 19:34:58 INFO com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystemBase: GHFS version: 1.6.1-hadoop2 17/08/22 19:34:58 INFO org.apache.hadoop.yarn.client.RMProxy: Connecting to ResourceManager at hadoop-m/10.240.1.92:8032<http://10.240.1.92:8032> 17/08/22 19:35:00 INFO org.apache.hadoop.yarn.client.api.impl.YarnClientImpl: Submitted application application_1503036971561_0022 17/08/22 19:35:04 WARN org.apache.spark.streaming.StreamingContext: Dynamic Allocation is enabled for this application. Enabling Dynamic allocation for Spark Streaming applications can cause data loss if Write Ahead Log is not enabled for non-replayable sources like Flume. See the programming guide for details on how to enable the Write Ahead Log. 17/08/22 19:35:21 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources 17/08/22 19:35:36 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources 17/08/22 19:35:51 WARN org.apache.spark.scheduler.cluster.YarnScheduler: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources I confirmed that the YARN cluster has enough memory for dozens of executors, and verified that the application allocates executors when using Core's spark.dynamicAllocation.enabled=true, and leaving spark.streaming.dynamicAllocation.enabled=false. Is streaming dynamic allocation actually supported? Sean Owen suggested it might have been experimental: https://issues.apache.org/jira/browse/SPARK-21792. -- Cheers!
Re: [Spark Streaming] Streaming Dynamic Allocation is broken (at least on YARN)
Have you tried setting spark.executor.instances=0 to a positive non-zero value? Also, since its a streaming application set executor cores > 1. On Wed, Aug 23, 2017 at 3:38 AM, Karthik Palaniappan wrote: > I ran the HdfsWordCount example using this command: > > spark-submit run-example \ > --conf spark.streaming.dynamicAllocation.enabled=true \ > --conf spark.executor.instances=0 \ > --conf spark.dynamicAllocation.enabled=false \ > --conf spark.master=yarn \ > --conf spark.submit.deployMode=client \ > org.apache.spark.examples.streaming.HdfsWordCount /foo > > I tried it on both Spark 2.1.1 (through HDP 2.6) and Spark 2.2.0 (through > Google Dataproc 1.2), and I get the same message repeatedly that Spark > cannot allocate any executors. > > 17/08/22 19:34:57 INFO org.spark_project.jetty.util.log: Logging > initialized @1694ms > 17/08/22 19:34:57 INFO org.spark_project.jetty.server.Server: > jetty-9.3.z-SNAPSHOT > 17/08/22 19:34:57 INFO org.spark_project.jetty.server.Server: Started > @1756ms > 17/08/22 19:34:57 INFO org.spark_project.jetty.server.AbstractConnector: > Started ServerConnector@578782d6{HTTP/1.1,[http/1.1]}{0.0.0.0:4040} > 17/08/22 19:34:58 INFO > com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystemBase: > GHFS version: 1.6.1-hadoop2 > 17/08/22 19:34:58 INFO org.apache.hadoop.yarn.client.RMProxy: Connecting > to ResourceManager at hadoop-m/10.240.1.92:8032 > 17/08/22 19:35:00 INFO org.apache.hadoop.yarn.client.api.impl.YarnClientImpl: > Submitted application application_1503036971561_0022 > 17/08/22 19:35:04 WARN org.apache.spark.streaming.StreamingContext: > Dynamic Allocation is enabled for this application. Enabling Dynamic > allocation for Spark Streaming applications can cause data loss if Write > Ahead Log is not enabled for non-replayable sources like Flume. See the > programming guide for details on how to enable the Write Ahead Log. > 17/08/22 19:35:21 WARN org.apache.spark.scheduler.cluster.YarnScheduler: > Initial job has not accepted any resources; check your cluster UI to ensure > that workers are registered and have sufficient resources > 17/08/22 19:35:36 WARN org.apache.spark.scheduler.cluster.YarnScheduler: > Initial job has not accepted any resources; check your cluster UI to ensure > that workers are registered and have sufficient resources > 17/08/22 19:35:51 WARN org.apache.spark.scheduler.cluster.YarnScheduler: > Initial job has not accepted any resources; check your cluster UI to ensure > that workers are registered and have sufficient resources > > I confirmed that the YARN cluster has enough memory for dozens of > executors, and verified that the application allocates executors when using > Core's spark.dynamicAllocation.enabled=true, and leaving spark.streaming. > dynamicAllocation.enabled=false. > > Is streaming dynamic allocation actually supported? Sean Owen suggested it > might have been experimental: https://issues.apache.org/jira/browse/SPARK- > 21792. > -- Cheers!