diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index fec05a76b4516..b83de0e39fb6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -242,7 +242,7 @@ object ShuffleExchangeExec { } // The comparator for comparing row hashcode, which should always be Integer. val prefixComparator = PrefixComparators.LONG - val canUseRadixSort = SQLConf.get.enableRadixSort + // The prefix computer generates row hashcode as the prefix, so we may decrease the // probability that the prefixes are equal when input rows choose column values from a // limited range. @@ -264,7 +264,7 @@ object ShuffleExchangeExec { prefixComparator, prefixComputer, pageSize, - canUseRadixSort) + false /* canUseRadixSort */) sorter.sort(iter.asInstanceOf[Iterator[UnsafeRow]]) } } else {