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

Broadcast join OutOfMemory stack trace obscures actual cause of OOM

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Minor
    • Resolution: Incomplete
    • 2.4.0
    • None
    • SQL

    Description

      When the Spark driver suffers an OutOfMemoryError while attempting to broadcast a table for a broadcast join, the resulting stack trace obscures the actual cause of the OOM. For e.g.:

      [GC (Allocation Failure)  585453K->585453K(928768K), 0.0060025 secs]
      [Full GC (Allocation Failure)  585453K->582524K(928768K), 0.4019639 secs]
      java.lang.OutOfMemoryError: Java heap space
      Dumping heap to java_pid12446.hprof ...
      Heap dump file created [632701033 bytes in 1.016 secs]
      Exception in thread "main" java.lang.OutOfMemoryError: Not enough memory to build and broadcast the table to all worker nodes. As a workaround, you can either disable broadcast by setting spark.sql.autoBroadcastJoinThreshold to -1 or increase the spark driver memory by setting spark.driver.memory to a higher value
      	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1$$anonfun$apply$1.apply(BroadcastExchangeExec.scala:122)
      	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1$$anonfun$apply$1.apply(BroadcastExchangeExec.scala:76)
      	at org.apache.spark.sql.execution.SQLExecution$$anonfun$withExecutionId$1.apply(SQLExecution.scala:101)
      	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
      	at org.apache.spark.sql.execution.SQLExecution$.withExecutionId(SQLExecution.scala:98)
      	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1.apply(BroadcastExchangeExec.scala:75)
      	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1.apply(BroadcastExchangeExec.scala:75)
      	at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
      	at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
      	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)
      18/07/24 14:29:58 INFO ContextCleaner: Cleaned accumulator 30
      18/07/24 14:29:58 INFO ContextCleaner: Cleaned accumulator 35
      

      The above stack trace blames BroadcastExchangeExec. However, the given line is actually where the original OutOfMemoryError was caught and a new one was created and wrapped by a SparkException. The actual location where the OOM occurred was in LongToUnsafeRowMap#grow, at this line:

      val newPage = new Array[Long](newNumWords.toInt)
      

      Sometimes it is helpful to know the actual location from which an OOM is thrown. In the above case, the location indicated that Spark underestimated the size of a large-ish table and ran out of memory trying to load it into memory.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              bersprockets Bruce Robbins
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: