alexr17 opened a new issue, #10311:
URL: https://github.com/apache/incubator-gluten/issues/10311
### Backend
VL (Velox)
### Bug description
## Build latest Gluten on July 21, 2025 and encountered a deadlock in
executors during shuffle operations.
I switched back to a build from June 12, 2025 and no longer hit this
deadlock. I am unable to RCA what is causing this.
What I observe:
- I am seeing consistent hang location always occurs at the same stage
during ColumnarShuffleExchangeExec
- CPU is running, but not maxed out, indicates threads are cycling.
- All executors become immediately blocked
- No Memory Pressure
- Thread dump from hung executor shows all task threads stuck in native code.
## Tried to dig through the code and found
When there are no leaf transformers, Gluten uses
WholeStageZippedPartitionsRDD. This calls genFinalStageIterator with multiple
input iterators (from Union). Creates WholeStageResultIterator which has an
infinite loop in its next() method:
```
while (true) {
auto future = velox::ContinueFuture::makeEmpty();
auto out = task_->next(&future);
if (!future.valid()) {
break;
}
future.wait(); // Appears to complete but task state doesn't change
}
```
The hang is in the native C++ code where:
- WholeStageResultIterator::next() calls into Velox
- The Velox task appears to be in a "busy" state but never progresses
- All CPU time seems to be spent in nativeHasNext() JNI calls
Problem is that I see no recent code changes which could have caused these
issues.
DAG of failed stage
<img width="851" height="875" alt="Image"
src="https://github.com/user-attachments/assets/45ca15c2-3e76-480a-9e00-9a5f2a64a195"
/>
DAG of same stage but successful with the earlier version of gluten
<img width="846" height="769" alt="Image"
src="https://github.com/user-attachments/assets/3ba73dee-a2f0-478a-92fa-4c09ae23091d"
/>
### Gluten version
main branch
### Spark version
Spark-3.5.x
### Spark configurations
Spark 3.5.2
### System information
Running Arm64 on Graviton2
### Relevant logs
```bash
"Executor task launch worker for task 11.0 in stage 15.0 (TID 1330)" #66
daemon prio=5 os_prio=0 cpu=2031451.92ms elapsed=3073.35s
tid=0x0000fffe04ccb120 nid=0x92 runnable [0x0000fffe6e1e9000]
java.lang.Thread.State: RUNNABLE
at
org.apache.gluten.vectorized.ColumnarBatchOutIterator.nativeHasNext(Native
Method)
at
org.apache.gluten.vectorized.ColumnarBatchOutIterator.hasNext0(ColumnarBatchOutIterator.java:57)
at
org.apache.gluten.iterator.ClosableIterator.hasNext(ClosableIterator.java:39)
at
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:45)
at
org.apache.gluten.iterator.IteratorsV1$InvocationFlowProtection.hasNext(IteratorsV1.scala:154)
at
org.apache.gluten.iterator.IteratorsV1$IteratorCompleter.hasNext(IteratorsV1.scala:66)
at
org.apache.gluten.iterator.IteratorsV1$PayloadCloser.hasNext(IteratorsV1.scala:38)
at
org.apache.gluten.iterator.IteratorsV1$LifeTimeAccumulator.hasNext(IteratorsV1.scala:95)
at
org.apache.gluten.iterator.IteratorsV1$ReadTimeAccumulator.hasNext(IteratorsV1.scala:122)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at
scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:32)
at
org.apache.gluten.vectorized.ColumnarBatchInIterator.hasNext(ColumnarBatchInIterator.java:36)
at
org.apache.gluten.vectorized.ColumnarBatchOutIterator.nativeHasNext(Native
Method)
at
org.apache.gluten.vectorized.ColumnarBatchOutIterator.hasNext0(ColumnarBatchOutIterator.java:57)
at
org.apache.gluten.iterator.ClosableIterator.hasNext(ClosableIterator.java:39)
at
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:45)
at
org.apache.gluten.iterator.IteratorsV1$InvocationFlowProtection.hasNext(IteratorsV1.scala:154)
at
org.apache.gluten.iterator.IteratorsV1$ReadTimeAccumulator.hasNext(IteratorsV1.scala:122)
at
org.apache.gluten.iterator.IteratorsV1$PayloadCloser.hasNext(IteratorsV1.scala:38)
at
org.apache.gluten.iterator.IteratorsV1$IteratorCompleter.hasNext(IteratorsV1.scala:66)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at
org.apache.spark.shuffle.ColumnarShuffleWriter.internalWrite(ColumnarShuffleWriter.scala:123)
at
org.apache.spark.shuffle.ColumnarShuffleWriter.write(ColumnarShuffleWriter.scala:277)
at
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:104)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
at
org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:166)
at org.apache.spark.scheduler.Task.run(Task.scala:141)
at
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620)
at
org.apache.spark.executor.Executor$TaskRunner$$Lambda$914/0x000000080142fc30.apply(Unknown
Source)
at
org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
at
org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623)
at
java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635)
at java.lang.Thread.run([email protected]/Thread.java:833)
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]