You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Dongjoon Hyun (Jira)" <ji...@apache.org> on 2020/09/14 19:25:00 UTC

[jira] [Created] (SPARK-32881) NoSuchElementException occurs during decommissioning

Dongjoon Hyun created SPARK-32881:
-------------------------------------

             Summary: NoSuchElementException occurs during decommissioning
                 Key: SPARK-32881
                 URL: https://issues.apache.org/jira/browse/SPARK-32881
             Project: Spark
          Issue Type: Bug
          Components: Spark Core
    Affects Versions: 3.1.0
            Reporter: Dongjoon Hyun


`BlockManagerMasterEndpoint` seems to fail at `getReplicateInfoForRDDBlocks` due to `java.util.NoSuchElementException`. This happens on K8s IT testing, but the main code seems to need a graceful handling of `NoSuchElementException` instead of showing a naive error message.
{code}
  private def getReplicateInfoForRDDBlocks(blockManagerId: BlockManagerId): Seq[ReplicateBlock] = {
    val info = blockManagerInfo(blockManagerId)
   ...
}
{code}
{code}
  20/09/14 18:56:54 INFO ExecutorPodsAllocator: Going to request 1 executors from Kubernetes.
  20/09/14 18:56:54 INFO BasicExecutorFeatureStep: Adding decommission script to lifecycle
  20/09/14 18:56:55 ERROR TaskSchedulerImpl: Lost executor 1 on 172.17.0.4: Executor decommission.
  20/09/14 18:56:55 INFO BlockManagerMaster: Removal of executor 1 requested
  20/09/14 18:56:55 INFO KubernetesClusterSchedulerBackend$KubernetesDriverEndpoint: Asked to remove non-existent executor 1
  20/09/14 18:56:55 INFO BlockManagerMasterEndpoint: Trying to remove executor 1 from BlockManagerMaster.
  20/09/14 18:56:55 INFO BlockManagerMasterEndpoint: Removing block manager BlockManagerId(1, 172.17.0.4, 41235, None)
  20/09/14 18:56:55 INFO DAGScheduler: Executor lost: 1 (epoch 1)
  20/09/14 18:56:55 ERROR Inbox: Ignoring error
  java.util.NoSuchElementException
  	at scala.collection.concurrent.TrieMap.apply(TrieMap.scala:833)
  	at org.apache.spark.storage.BlockManagerMasterEndpoint.org$apache$spark$storage$BlockManagerMasterEndpoint$$getReplicateInfoForRDDBlocks(BlockManagerMasterEndpoint.scala:383)
  	at org.apache.spark.storage.BlockManagerMasterEndpoint$$anonfun$receiveAndReply$1.applyOrElse(BlockManagerMasterEndpoint.scala:171)
  	at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:103)
  	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:203)
  	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
  	at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75)
  	at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41)
  	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
  	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
  	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
  	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
  	at java.lang.Thread.run(Thread.java:748)
  20/09/14 18:56:55 INFO BlockManagerMasterEndpoint: Trying to remove executor 1 from BlockManagerMaster.
  20/09/14 18:56:55 INFO BlockManagerMaster: Removed 1 successfully in removeExecutor
  20/09/14 18:56:55 INFO DAGScheduler: Shuffle files lost for executor: 1 (epoch 1)
  20/09/14 18:56:58 INFO KubernetesClusterSchedulerBackend$KubernetesDriverEndpoint: Registered executor NettyRpcEndpointRef(spark-client://Executor) (172.17.0.7:46674) with ID 4,  ResourceProfileId 0
  20/09/14 18:56:58 INFO BlockManagerMasterEndpoint: Registering block manager 172.17.0.7:40495 with 593.9 MiB RAM, BlockManagerId(4, 172.17.0.7, 40495, None)
  20/09/14 18:57:23 INFO SparkContext: Starting job: count at /opt/spark/tests/decommissioning.py:49
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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