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

decom.sh can cause an UnsupportedOperationException

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 3.3.0
    • None
    • Kubernetes
    • None

    Description

      decom.sh can cause an UnsupportedOperationException which then causes the Executor to die with a SparkUncaughtException and does not complete the decommission properly.

       
      Problem:
      SignalUtils.scala line 124:
       

      if (escalate) {
         prevHandler.handle(sig)
      }

       
       
      Logs:
       

      failed - error: command '/opt/decom.sh' exited with 137: + echo 'Asked to decommission' + date + tee -a ++ ps -o pid -C java ++ awk '{ sub(/^[ \t]+/, """"); print }' ++ tail -n 1 + WORKER_PID=17 + echo 'Using worker pid 17' + kill -s SIGPWR 17 + echo 'Waiting for worker pid to exit' + timeout 60 tail --pid=17 -f /dev/null , message: ""Asked to decommission\nMon Apr 17 23:44:35 UTC 2023\nUsing worker pid 17\nWaiting for worker pid to exit\n+ echo 'Asked to decommission'\n+ date\n+ tee -a\n++ ps -o pid -C java\n++ awk '{ sub(/^[ \\t]+/, \""\""); print }'\n++ tail -n 1\n+ WORKER_PID=17\n+ echo 'Using worker pid 17'\n+ kill -s SIGPWR 17\n+ echo 'Waiting for worker pid to exit'\n+ timeout 60 tail --pid=17 -f /dev/null\n""",2023-04-17T23:44:39Z,
      "java.lang.UnsupportedOperationException: invoking native signal handle not supported
       at java.base/jdk.internal.misc.Signal$NativeHandler.handle(Unknown Source)
       at jdk.unsupported/sun.misc.Signal$SunMiscHandler.handle(Unknown Source)
       at org.apache.spark.util.SignalUtils$ActionHandler.handle(SignalUtils.scala:124)
       at jdk.unsupported/sun.misc.Signal$InternalMiscHandler.handle(Unknown Source)
       at java.base/jdk.internal.misc.Signal$1.run(Unknown Source) at java.base/java.lang.Thread.run(Unknown Source)",2023-04-17T23:44:35.407488217Z "2023-04-17 23:44:35
      [SIGPWR handler] ERROR org.apache.spark.util.SparkUncaughtExceptionHandler - Uncaught exception in thread Thread[SIGPWR handler,9,system] - {}",2023-04-17T23:44:35.407457859Z
       " ... 1 more",2023-04-17T23:44:35.405548994Z "
       at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)",2023-04-17T23:44:35.405542621Z "
       at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)",2023-04-17T23:44:35.405536674Z "
       at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)",2023-04-17T23:44:35.405516396Z "
       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:496)",2023-04-17T23:44:35.405416352Z "
       at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:584)",2023-04-17T23:44:35.405410491Z "
      ...
       at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:53)",2023-04-17T23:44:35.405262304Z "
       at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:142)",2023-04-17T23:44:35.405256591Z "
       at org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:209)",2023-04-17T23:44:35.405250814Z

       

      In this case prevHandler is the NativeHandler (See https://github.com/AdoptOpenJDK/openjdk-jdk11/blob/19fb8f93c59dfd791f62d41f332db9e306bc1422/src/java.base/share/classes/jdk/internal/misc/Signal.java#L280) and it throws the exception.

      Possible Solutions:

      • Check if prevHandler is an instance of NativeHandler and do not call it in that case.
      • try catch around the invoke of the handler and log a warning/error on exceptions.

      Attachments

        Activity

          People

            Unassigned Unassigned
            icardnell Iain Cardnell
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: