Skip to content

Commit 2d9cc42

Browse files
xuanyuankingdongjoon-hyun
authored andcommitted
[SPARK-28699][SQL] Disable using radix sort for ShuffleExchangeExec in repartition case
## What changes were proposed in this pull request? Disable using radix sort in ShuffleExchangeExec when we do repartition. In #20393, we fixed the indeterministic result in the shuffle repartition case by performing a local sort before repartitioning. But for the newly added sort operation, we use radix sort which is wrong because binary data can't be compared by only the prefix. This makes the sort unstable and fails to solve the indeterminate shuffle output problem. ### Why are the changes needed? Fix the correctness bug caused by repartition after a shuffle. ### Does this PR introduce any user-facing change? Yes, user will get the right result in the case of repartition stage rerun. ## How was this patch tested? Test with `local-cluster[5, 2, 5120]`, use the integrated test below, it can return a right answer 100000000. ``` import scala.sys.process._ import org.apache.spark.TaskContext val res = spark.range(0, 10000 * 10000, 1).map{ x => (x % 1000, x)} // kill an executor in the stage that performs repartition(239) val df = res.repartition(113).map{ x => (x._1 + 1, x._2)}.repartition(239).map { x => if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 1 && TaskContext.get.stageAttemptNumber == 0) { throw new Exception("pkill -f -n java".!!) } x } val r2 = df.distinct.count() ``` Closes #25491 from xuanyuanking/SPARK-28699-fix. Authored-by: Yuanjian Li <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
1 parent 4dc3093 commit 2d9cc42

File tree

1 file changed

+4
-2
lines changed

1 file changed

+4
-2
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -242,7 +242,7 @@ object ShuffleExchangeExec {
242242
}
243243
// The comparator for comparing row hashcode, which should always be Integer.
244244
val prefixComparator = PrefixComparators.LONG
245-
val canUseRadixSort = SQLConf.get.enableRadixSort
245+
246246
// The prefix computer generates row hashcode as the prefix, so we may decrease the
247247
// probability that the prefixes are equal when input rows choose column values from a
248248
// limited range.
@@ -264,7 +264,9 @@ object ShuffleExchangeExec {
264264
prefixComparator,
265265
prefixComputer,
266266
pageSize,
267-
canUseRadixSort)
267+
// We are comparing binary here, which does not support radix sort.
268+
// See more details in SPARK-28699.
269+
false)
268270
sorter.sort(iter.asInstanceOf[Iterator[UnsafeRow]])
269271
}
270272
} else {

0 commit comments

Comments
 (0)