[jira] [Commented] (SPARK-33447) Stage level scheduling, allow specifying other spark configs via ResourceProfile

2023-11-17 Thread Jeroen Dries (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-33447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17787455#comment-17787455
 ] 

Jeroen Dries commented on SPARK-33447:
--

Another use case: we would like to modify speculation settings on a per-stage 
basis.

> Stage level scheduling, allow specifying other spark configs via 
> ResourceProfile
> 
>
> Key: SPARK-33447
> URL: https://issues.apache.org/jira/browse/SPARK-33447
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core
>Affects Versions: 3.1.0
>Reporter: Thomas Graves
>Priority: Major
>
> With the addition of stage level scheduling and ResourceProfiles, we only 
> allow certain resources to specified currently.  There are many other configs 
> that users may want to change between changes. We should look at adding in 
> perhaps SparkConf settings.
> One very specific one that was brought up in review was ability to change 
> yarn queue (if that is even possible in yarn) between stages because someone 
> might want ot use one queue for ETL and a separate queue for ML where there 
> are nodes with GPU.  Or perhaps node labels separately.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-39717) Spark on YARN + preemption: "Not removing executor 304 because the ResourceProfile was UNKNOWN"

2022-07-08 Thread Jeroen Dries (Jira)
Jeroen Dries created SPARK-39717:


 Summary: Spark on YARN + preemption: "Not removing executor 304 
because the ResourceProfile was UNKNOWN"
 Key: SPARK-39717
 URL: https://issues.apache.org/jira/browse/SPARK-39717
 Project: Spark
  Issue Type: Bug
  Components: Spark Core
Affects Versions: 3.2.1
 Environment: YARN Hadoop 3.2 cluster

YARN_CONTAINER_RUNTIME_TYPE=docker

spark.speculation=true

spark.dynamicAllocation.enabled=true

 
Reporter: Jeroen Dries


The Spark driver continuously logs these messages for about 4 executors, after 
preemption of these executors. The spark app is still functional, and many more 
containers have been preempted without resulting in these warnings.

 

When looking at the spark code, it seems that this is an unexpected state. The 
spark UI also no longer shows these executors.

Restarting the application makes it go away for some time, but it always comes 
back.

 
{code:java}
{"lineno":69,"name":"org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Requesting
 driver to remove executor 304 for reason Container cont
ainer_e5047_1656725539918_7361_01_001553 on host: epod062.vgt.vito.be was 
preempted.","created":1657198251.106}

{"lineno":73,"name":"org.apache.spark.scheduler.cluster.YarnScheduler","process":31,"filename":"Logging.scala","levelname":"ERROR","message":"Lost
 executor 304 on epod062.vgt.vito.be: Container 
container_e5047_1656725539918_7361_01_0015
53 on host: epod062.vgt.vito.be was preempted.","created":1657198251.111}


{code}
{code:java}
 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 287 because the ResourceProfile was 
UNKNOWN!","created":1657198313.857} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 296 because the ResourceProfile was 
UNKNOWN!","created":1657198313.858} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 295 because the ResourceProfile was 
UNKNOWN!","created":1657198313.858} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 304 because the ResourceProfile was 
UNKNOWN!","created":1657198313.858} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 287 because the ResourceProfile was 
UNKNOWN!","created":1657198313.959} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 296 because the ResourceProfile was 
UNKNOWN!","created":1657198313.959} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 295 because the ResourceProfile was 
UNKNOWN!","created":1657198313.959} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 304 because the ResourceProfile was 
UNKNOWN!","created":1657198313.959} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 287 because the ResourceProfile was 
UNKNOWN!","created":1657198314.059} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 296 because the ResourceProfile was 
UNKNOWN!","created":1657198314.059} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 295 because the ResourceProfile was 
UNKNOWN!","created":1657198314.059} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 304 because the ResourceProfile was 
UNKNOWN!","created":1657198314.059} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 287 because the ResourceProfile was 
UNKNOWN!","created":1657198314.160} 
{"lineno":69,"name":"org.apache.spark.ExecutorAllocationManager","process":31,"filename":"Logging.scala","levelname":"WARN","message":"Not
 removing executor 296 because the ResourceProfile was 
UNKNOWN!","created":1657198314.160} 

[jira] [Commented] (SPARK-29965) Race in executor shutdown handling can lead to executor never fully unregistering

2020-07-15 Thread Jeroen Dries (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-29965?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17157936#comment-17157936
 ] 

Jeroen Dries commented on SPARK-29965:
--

We seem to see the same (or similar?) issue on Spark 2.4.4. We have a long 
running application, running on a yarn (hortonworks 3.1.4.0-315) cluster with 
dynamic allocation and preemption enabled.

After a fresh restart, logs are fine, but a few days later, our logs are being 
spammed with messages like this:
{code:java}
20/07/15 08:34:10 WARN BlockManagerMasterEndpoint: Error trying to remove 
broadcast 258143 from block manager BlockManagerId(10469, epod036.vgt.vito.be, 
35676, None)
java.io.IOException: Failed to send RPC RPC 8615372230556941680 to 
/192.168.207.136:51296: java.nio.channels.ClosedChannelException
at 
org.apache.spark.network.client.TransportClient$RpcChannelListener.handleFailure(TransportClient.java:362)
at 
org.apache.spark.network.client.TransportClient$StdChannelListener.operationComplete(TransportClient.java:339)
at 
io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:507)
at 
io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:481)
at 
io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:420)
at 
io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:122)
at 
io.netty.channel.AbstractChannel$AbstractUnsafe.safeSetFailure(AbstractChannel.java:987)
at 
io.netty.channel.AbstractChannel$AbstractUnsafe.write(AbstractChannel.java:869)
at 
io.netty.channel.DefaultChannelPipeline$HeadContext.write(DefaultChannelPipeline.java:1316)
at 
io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:738)
at 
io.netty.channel.AbstractChannelHandlerContext.invokeWrite(AbstractChannelHandlerContext.java:730)
at 
io.netty.channel.AbstractChannelHandlerContext.access$1900(AbstractChannelHandlerContext.java:38)
at 
io.netty.channel.AbstractChannelHandlerContext$AbstractWriteTask.write(AbstractChannelHandlerContext.java:1081)
at 
io.netty.channel.AbstractChannelHandlerContext$WriteAndFlushTask.write(AbstractChannelHandlerContext.java:1128)
at 
io.netty.channel.AbstractChannelHandlerContext$AbstractWriteTask.run(AbstractChannelHandlerContext.java:1070)
at 
io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
at 
io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403)
at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:463)
at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.nio.channels.ClosedChannelException
at io.netty.channel.AbstractChannel$AbstractUnsafe.write(...)(Unknown 
Source)
 {code}

> Race in executor shutdown handling can lead to executor never fully 
> unregistering
> -
>
> Key: SPARK-29965
> URL: https://issues.apache.org/jira/browse/SPARK-29965
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 3.0.0
>Reporter: Marcelo Masiero Vanzin
>Priority: Major
>
> I ran into a situation that I had never noticed before, but I seem to be able 
> to hit with just a few retries when using K8S with dynamic allocation.
> Basically, there's a race when killing an executor, where it may send a 
> heartbeat to the driver right at the wrong time during shutdown, e.g.:
> {noformat}
> 19/11/19 21:14:05 INFO CoarseGrainedExecutorBackend: Driver commanded a 
> shutdown
> 19/11/19 21:14:05 INFO Executor: Told to re-register on heartbeat
> 19/11/19 21:14:05 INFO BlockManager: BlockManager BlockManagerId(10, 
> 192.168.3.99, 39923, None) re-registering with master
> 19/11/19 21:14:05 INFO BlockManagerMaster: Registering BlockManager 
> BlockManagerId(10, 192.168.3.99, 39923, None)
> 19/11/19 21:14:05 INFO BlockManagerMaster: Registered BlockManager 
> BlockManagerId(10, 192.168.3.99, 39923, None)
> 19/11/19 21:14:06 INFO BlockManager: Reporting 0 blocks to the master.
> {noformat}
> On the driver side it will happily re-register the executor (time diff is 
> just because of time zone in log4j config):
> {noformat}
> 19/11/19 13:14:05 INFO BlockManagerMasterEndpoint: Trying to remove executor 
> 10 from BlockManagerMaster.
> 19/11/19 13:14:05 INFO BlockManagerMasterEndpoint: Removing block manager 
> BlockManagerId(10, 192.168.3.99, 39923, None)
> 19/11/19 13:14:05 INFO BlockManagerMaster: