diff --git a/spark/src/main/scala/org/apache/comet/CometConf.scala b/spark/src/main/scala/org/apache/comet/CometConf.scala index af8fbb5815..9d0b319276 100644 --- a/spark/src/main/scala/org/apache/comet/CometConf.scala +++ b/spark/src/main/scala/org/apache/comet/CometConf.scala @@ -273,7 +273,7 @@ object CometConf extends ShimCometConf { val COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("takeOrderedAndProject", defaultValue = true) val COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED: ConfigEntry[Boolean] = - createExecEnabledConfig("localTableScan", defaultValue = false) + createExecEnabledConfig("localTableScan", defaultValue = true) val COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.columnarToRow.native.enabled") diff --git a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala index 544cd91bd2..a9fdc96231 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala @@ -108,15 +108,7 @@ class CometWindowExecSuite extends CometTestBase { val cometShuffles = collect(df2.queryExecution.executedPlan) { case _: CometShuffleExchangeExec => true } - if (shuffleMode == "jvm" || shuffleMode == "auto") { - assert(cometShuffles.length == 1) - } else { - // we fall back to Spark for shuffle because we do not support - // native shuffle with a LocalTableScan input, and we do not fall - // back to Comet columnar shuffle due to - // https://github.com/apache/datafusion-comet/issues/1248 - assert(cometShuffles.isEmpty) - } + assert(cometShuffles.length == 1) } } }