Marcelo Masiero Vanzin created SPARK-29965: ----------------------------------------------
Summary: 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 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: Removed 10 successfully in removeExecutor 19/11/19 13:14:05 INFO DAGScheduler: Shuffle files lost for executor: 10 (epoch 18) {noformat} And a little later: {noformat} 19/11/19 13:14:05 DEBUG HeartbeatReceiver: Received heartbeat from unknown executor 10 19/11/19 13:14:05 INFO BlockManagerMasterEndpoint: Registering block manager 192.168.3.99:39923 with 413.9 MiB RAM, BlockManagerId(10, 192.168.3.99, 39923, None) {noformat} This becomes a problem later, where you start to see period exceptions in the driver's logs: {noformat} 19/11/19 13:14:39 WARN BlockManagerMasterEndpoint: Error trying to remove broadcast 4 from block manager BlockManagerId(10, 192.168.3.99, 39923, None) java.io.IOException: Failed to send RPC RPC 4999007301825869809 to /10.65.55.240:14233: 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:577) at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:551) {noformat} That happens every time some code calls into the block manager to request stuff from all executors. Meaning that the dead executor re-registered, and then was never removed from the block manager. I found a few races in the code that can lead to this situation. I'll post a PR once I test it more. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org