Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-32881

NoSuchElementException occurs during decommissioning

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.1.0
    • 3.1.0
    • Spark Core
    • None

    Description

      `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.

      private def getReplicateInfoForRDDBlocks(blockManagerId: BlockManagerId): Seq[ReplicateBlock] = {
          val info = blockManagerInfo(blockManagerId)
         ...
      }
      
        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
      

      Attachments

        Activity

          People

            holden Holden Karau
            dongjoon Dongjoon Hyun
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: