@@ -139,8 +139,44 @@ class CometExecSuite extends CometTestBase {
139139 val (_, cometPlan) = checkSparkAnswer(df)
140140 val infos = new ExtendedExplainInfo ().generateExtendedInfo(cometPlan)
141141 assert(infos.contains(" Dynamic Partition Pruning is not supported" ))
142+ }
143+ }
144+ }
145+ }
146+
147+ test(" DPP fallback avoids inefficient Comet shuffle (#3874)" ) {
148+ withTempDir { path =>
149+ val factPath = s " ${path.getAbsolutePath}/fact.parquet "
150+ val dimPath = s " ${path.getAbsolutePath}/dim.parquet "
151+ withSQLConf(CometConf .COMET_EXEC_ENABLED .key -> " false" ) {
152+ val one_day = 24 * 60 * 60000
153+ val fact = Range (0 , 100 )
154+ .map(i => (i, new java.sql.Date (System .currentTimeMillis() + i * one_day), i.toString))
155+ .toDF(" fact_id" , " fact_date" , " fact_str" )
156+ fact.write.partitionBy(" fact_date" ).parquet(factPath)
157+ val dim = Range (0 , 10 )
158+ .map(i => (i, new java.sql.Date (System .currentTimeMillis() + i * one_day), i.toString))
159+ .toDF(" dim_id" , " dim_date" , " dim_str" )
160+ dim.write.parquet(dimPath)
161+ }
162+
163+ // Force sort-merge join to get a shuffle exchange above the DPP scan
164+ Seq (" parquet" ).foreach { v1List =>
165+ withSQLConf(
166+ SQLConf .USE_V1_SOURCE_LIST .key -> v1List,
167+ SQLConf .AUTO_BROADCASTJOIN_THRESHOLD .key -> " -1" ,
168+ CometConf .COMET_DPP_FALLBACK_ENABLED .key -> " true" ) {
169+ spark.read.parquet(factPath).createOrReplaceTempView(" dpp_fact2" )
170+ spark.read.parquet(dimPath).createOrReplaceTempView(" dpp_dim2" )
171+ val df =
172+ spark.sql(
173+ " select * from dpp_fact2 join dpp_dim2 on fact_date = dim_date where dim_id > 7" )
174+ val (_, cometPlan) = checkSparkAnswer(df)
142175
143- assert(infos.contains(" Comet accelerated" ))
176+ // Verify no CometShuffleExchangeExec wraps the DPP stage
177+ assert(
178+ ! cometPlan.toString().contains(" CometColumnarShuffle" ),
179+ " Should not use Comet columnar shuffle for stages with DPP scans" )
144180 }
145181 }
146182 }
0 commit comments