Skip to content

Conversation

@AngersZhuuuu
Copy link
Contributor

@AngersZhuuuu AngersZhuuuu commented Dec 23, 2025

What changes were proposed in this pull request?

These days I am checking cluster's OOM failed APP, I found allocate failed didn't log out the error stack. A little not friendly for user check app.

Such as user set 200M broadcast threshold, but allocate 4G memory

25/12/21 07:08:13 WARN [broadcast-exchange-4] TaskMemoryManager: Failed to allocate a page (4294967296 bytes), try again.
25/12/21 07:08:58 WARN [broadcast-exchange-4] TaskMemoryManager: Failed to allocate a page (4294967296 bytes), try again.

Why are the changes needed?

Help user debug

Does this PR introduce any user-facing change?

User can know allocate what memory failed

How was this patch tested?

Before

11:45:10.693 WARN org.apache.spark.memory.TaskMemoryManager: Failed to allocate a page (67108848 bytes), try again.

After

11:45:10.693 WARN org.apache.spark.memory.TaskMemoryManager: Failed to allocate a page (67108848 bytes), try again.
java.lang.OutOfMemoryError: Failed to allocate 67108848
        at org.apache.spark.unsafe.memory.HeapMemoryAllocator.allocate(HeapMemoryAllocator.java:49)
        at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:398)
        at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:359)
        at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:118)
        at org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch.acquirePage(RowBasedKeyValueBatch.java:129)
        at org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch.<init>(RowBasedKeyValueBatch.java:108)
        at org.apache.spark.sql.catalyst.expressions.FixedLengthRowBasedKeyValueBatch.<init>(FixedLengthRowBasedKeyValueBatch.java:1    at org.apache.spark.scheduler.Task.run(Task.scala:147)
        at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:716)
        at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:86)
        at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:83)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:97)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:719)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
        at java.base/java.lang.Thread.run(Thread.java:842)
java.lang.OutOfMemoryError: Java heap space
        at org.apache.spark.unsafe.memory.HeapMemoryAllocator.allocate(HeapMemoryAllocator.java:49)
        at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:398)
        at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:359)
        at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:118)
        at org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch.acquirePage(RowBasedKeyValueBatch.java:129)
        at org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch.<init>(RowBasedKeyValueBatch.java:108)
        at org.apache.spark.sql.catalyst.expressions.FixedLengthRowBasedKeyValueBatch.<init>(FixedLengthRowBasedKeyValueBatch.java:169)
        at org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch.allocate(RowBasedKeyValueBatch.java:91)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$hashAgg_FastHashMap_0.<init>(Unknown Source)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
        at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:50)
        at scala.collection.Iterator$$anon$9.hasNext(Iterator.scala:593)
        at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:153)
        at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:57)
        at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:111)
        at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
        at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:180)
        at org.apache.spark.scheduler.Task.run(Task.scala:147)
        at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:716)
        at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:86)
        at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:83)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:97)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:719)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
        at java.base/java.lang.Thread.run(Thread.java:842)

Was this patch authored or co-authored using generative AI tooling?

No

@github-actions github-actions bot added the CORE label Dec 23, 2025
@AngersZhuuuu
Copy link
Contributor Author

gentle ping @cloud-fan @dongjoon-hyun WDYT?

public MemoryBlock allocatePage(
long size,
MemoryConsumer consumer,
boolean isRetry) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add. this avoid print too much stack information.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

given we are touching this place, can we use int retryCount instead of a boolean? And we can also log the retry count.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done, how about current code.

* @throws TooLargePageException
*/
public MemoryBlock allocatePage(long size, MemoryConsumer consumer) {
public MemoryBlock allocatePage(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can this be private now?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants