parthchandra commented on code in PR #3982:
URL: https://github.com/apache/datafusion-comet/pull/3982#discussion_r3103583582


##########
spark/src/main/scala/org/apache/comet/CometFallback.scala:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.comet
+
+import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag}
+
+import org.apache.comet.CometSparkSessionExtensions.withInfo
+
+/**
+ * Sticky fallback marker for shuffle / stage nodes.
+ *
+ * Comet's shuffle-support predicates (e.g. 
`CometShuffleExchangeExec.columnarShuffleSupported`)
+ * run at both initial planning and AQE stage-prep. Some fallback decisions 
depend on the
+ * surrounding plan shape - for example, the presence of a DPP scan below a 
shuffle. Between the
+ * two passes AQE can reshape that subtree (a completed child stage becomes a
+ * `ShuffleQueryStageExec`, a `LeafExecNode` whose `children` is empty), so a 
naive re-evaluation
+ * can flip the decision.
+ *
+ * When a decision is made on the initial-plan pass, the deciding rule records 
a sticky tag via
+ * [[markForFallback]]. On subsequent passes, callers short-circuit via 
[[isMarkedForFallback]]
+ * and preserve the earlier decision instead of re-deriving it from the 
current plan shape.
+ *
+ * This tag is kept separate from `CometExplainInfo.EXTENSION_INFO` on 
purpose: the explain tag
+ * accumulates informational reasons (including rolled-up child reasons), many 
of which are not a
+ * full-fallback signal. Treating any presence of explain info as fallback is 
too coarse and
+ * breaks legitimate conversions (e.g. a shuffle tagged "Comet native shuffle 
not enabled" should
+ * still be eligible for columnar shuffle). The fallback tag exists only for 
decisions that should
+ * remain sticky.
+ */
+object CometFallback {
+
+  val STAGE_FALLBACK_TAG: TreeNodeTag[Set[String]] =
+    new TreeNodeTag[Set[String]]("CometStageFallback")
+
+  /**
+   * Mark a node so that subsequent shuffle-support re-evaluations fall back 
to Spark without
+   * re-deriving the decision from the (possibly reshaped) subtree. Also 
records the reason in the
+   * usual explain channel so it surfaces in extended explain output.
+   */
+  def markForFallback[T <: TreeNode[_]](node: T, reason: String): T = {

Review Comment:
   This is much better than using explain info



##########
spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala:
##########
@@ -450,12 +456,17 @@ object CometShuffleExchangeExec
         false
     }
 
+    // Preserve any prior-pass fallback decision (see `CometFallback`).
+    if (isMarkedForFallback(s)) {
+      return false
+    }
+
     if (!isCometShuffleEnabledWithInfo(s)) {
       return false
     }
 
     if (CometConf.COMET_DPP_FALLBACK_ENABLED.get() && stageContainsDPPScan(s)) 
{
-      withInfo(s, "Stage contains a scan with Dynamic Partition Pruning")
+      markForFallback(s, "Stage contains a scan with Dynamic Partition 
Pruning")

Review Comment:
   Should we add the reason to explainInfo? 



##########
spark/src/test/scala/org/apache/spark/sql/comet/CometDppFallbackRepro3949Suite.scala:
##########
@@ -0,0 +1,430 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.comet
+
+import scala.collection.mutable
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{CometTestBase, Row}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, PlanExpression}
+import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec
+import org.apache.spark.sql.execution.{FileSourceScanExec, LeafExecNode, 
SparkPlan}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.internal.SQLConf
+
+import org.apache.comet.{CometConf, CometExplainInfo}
+
+/**
+ * Attempts an end-to-end reproduction of issue #3949.
+ *
+ * #3949 reports `[INTERNAL_ERROR]` from 
`BroadcastExchangeExec.doCanonicalize`, ultimately caused
+ * by `ColumnarToRowExec.<init>` asserting that its child `supportsColumnar`. 
The suspected root
+ * cause is that Comet's DPP fallback 
(`CometShuffleExchangeExec.stageContainsDPPScan`) walks
+ * `s.child.exists(...)` for a `FileSourceScanExec` with a `PlanExpression` 
partition filter, and
+ * that walk is not stable across the two planning passes:
+ *
+ *   - initial planning: shuffle's child subtree includes the DPP scan -> 
`.exists` finds it ->
+ *     fall back to Spark.
+ *   - AQE stage prep: the inner stage that contained the scan has 
materialized and been replaced
+ *     by a `ShuffleQueryStageExec` (a `LeafExecNode` whose `children == 
Seq.empty`). `.exists`
+ *     can no longer descend into it, the DPP scan becomes invisible, the same 
shuffle is
+ *     converted to Comet, and the plan shape changes between passes.
+ *
+ * This suite has two tests:
+ *
+ *   1. `mechanism`: synthetic. Builds a real DPP plan, observes the 
initial-pass decision is
+ *      "fall back", then swaps the shuffle's child for an opaque 
`LeafExecNode` (mirroring how
+ *      `ShuffleQueryStageExec` presents to `.exists`) and asserts the 
decision flips to
+ *      "convert". Documents the mechanism without depending on AQE actually 
triggering it.
+ *
+ * 2. `endToEnd`: runs DPP-flavored queries with AQE on, sweeps a few 
seeds/variants, and asks
+ * whether `df.collect()` ever throws or whether the final executed plan ever 
contains a Comet
+ * shuffle whose child subtree (descending through `QueryStageExec.plan`) 
still contains a DPP
+ * scan -- i.e. an inconsistency that the bug would produce.
+ */
+class CometDppFallbackRepro3949Suite extends CometTestBase {
+
+  // ----------------------------------------------------------------------
+  // Mechanism (synthetic): proves the AQE wrap flips the fallback decision.
+  // ----------------------------------------------------------------------
+
+  private def buildDppTables(dir: java.io.File, factPrefix: String): Unit = {
+    val factPath = s"${dir.getAbsolutePath}/$factPrefix.parquet"
+    val dimPath = s"${dir.getAbsolutePath}/${factPrefix}_dim.parquet"
+    withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "false") {
+      val sess = spark
+      import sess.implicits._
+      val oneDay = 24L * 60L * 60000L
+      val now = System.currentTimeMillis()
+      (0 until 400)
+        .map(i => (i, new java.sql.Date(now + (i % 40) * oneDay), i.toString))
+        .toDF("fact_id", "fact_date", "fact_str")
+        .write
+        .partitionBy("fact_date")
+        .parquet(factPath)
+      (0 until 40)
+        .map(i => (i, new java.sql.Date(now + i * oneDay), i.toString))
+        .toDF("dim_id", "dim_date", "dim_str")
+        .write
+        .parquet(dimPath)
+    }
+    spark.read.parquet(factPath).createOrReplaceTempView(s"${factPrefix}_fact")
+    spark.read.parquet(dimPath).createOrReplaceTempView(s"${factPrefix}_dim")
+  }
+
+  private def unwrapAqe(plan: SparkPlan): SparkPlan = plan match {
+    case a: AdaptiveSparkPlanExec => a.initialPlan
+    case other => other
+  }
+
+  private def findFirstShuffle(plan: SparkPlan): Option[ShuffleExchangeExec] = 
{
+    var found: Option[ShuffleExchangeExec] = None
+    plan.foreach {
+      case s: ShuffleExchangeExec if found.isEmpty => found = Some(s)
+      case _ =>
+    }
+    found
+  }
+
+  test("mechanism: DPP fallback decision is sticky across an AQE-style child 
wrap") {
+    withTempDir { dir =>
+      buildDppTables(dir, "mech")
+      withSQLConf(
+        CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true",
+        SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
+        SQLConf.PREFER_SORTMERGEJOIN.key -> "true",
+        SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true",
+        SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") {
+
+        val df = spark.sql(
+          "select f.fact_date, count(*) c " +
+            "from mech_fact f join mech_dim d on f.fact_date = d.dim_date " +
+            "where d.dim_id > 35 group by f.fact_date")
+        val initialPlan = unwrapAqe(df.queryExecution.executedPlan)
+        val shuffle = findFirstShuffle(initialPlan).getOrElse {
+          fail(s"No ShuffleExchangeExec found in initial 
plan:\n${initialPlan.treeString}")
+        }
+
+        val initialDecision = 
CometShuffleExchangeExec.columnarShuffleSupported(shuffle)
+
+        val initialDppVisible = shuffle.child.exists {
+          case scan: FileSourceScanExec =>
+            
scan.partitionFilters.exists(_.exists(_.isInstanceOf[PlanExpression[_]]))
+          case _ => false
+        }
+
+        // Simulate AQE stage prep: wrap the shuffle's child in an opaque 
LeafExecNode,
+        // matching how `ShuffleQueryStageExec` presents to `.exists` walks 
(its `children`
+        // is `Seq.empty`). `withNewChildren` preserves tree-node tags, so if 
the fix is in
+        // place the sticky CometFallback marker on `shuffle` carries over to
+        // `postAqeShuffle`, and the decision short-circuits to false. Without 
the fix,
+        // the DPP walk re-runs, fails to see the scan, and flips to true.
+        val hiddenChild = OpaqueStageStub(shuffle.child.output)
+        val postAqeShuffle =
+          
shuffle.withNewChildren(Seq(hiddenChild)).asInstanceOf[ShuffleExchangeExec]
+        val postAqeDecision = 
CometShuffleExchangeExec.columnarShuffleSupported(postAqeShuffle)
+
+        val postAqeDppVisible = postAqeShuffle.child.exists {
+          case scan: FileSourceScanExec =>
+            
scan.partitionFilters.exists(_.exists(_.isInstanceOf[PlanExpression[_]]))
+          case _ => false
+        }
+
+        // scalastyle:off println
+        println("=== mechanism check ===")

Review Comment:
   Use `log` insted of `println`, or perhaps you meant to remove this?



-- 
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]

Reply via email to