Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,19 +35,34 @@ object NormalizeCTEIds extends Rule[LogicalPlan] {
plan: LogicalPlan,
curId: AtomicLong,
cteIdToNewId: mutable.Map[Long, Long]): LogicalPlan = {
plan transformDownWithSubqueries {
val newIdValues = mutable.Set.empty[Long]
val normalized = plan transformDownWithSubqueries {
case ctas @ CacheTableAsSelect(_, plan, _, _, _, _, _) =>
ctas.copy(plan = applyInternal(plan, curId, cteIdToNewId))

case withCTE @ WithCTE(plan, cteDefs) =>
val newCteDefs = cteDefs.map { cteDef =>
cteIdToNewId.getOrElseUpdate(cteDef.id, curId.getAndIncrement())
newIdValues += cteIdToNewId(cteDef.id)
val normalizedCteDef = canonicalizeCTE(cteDef.child, cteIdToNewId)
cteDef.copy(child = normalizedCteDef, id = cteIdToNewId(cteDef.id))
}
val normalizedPlan = canonicalizeCTE(plan, cteIdToNewId)
withCTE.copy(plan = normalizedPlan, cteDefs = newCteDefs)
}
// SPARK-56739: Second pass to normalize orphan CTERelationRefs that exist outside any
// WithCTE node (e.g., after InlineCTE or MergeSubplans removes the parent WithCTE).
// Skip refs whose cteId is already a normalized value (to avoid double-normalization
// when original IDs and new IDs overlap).
if (cteIdToNewId.nonEmpty) {
normalized.transformDownWithSubqueries {
case ref: CTERelationRef
if cteIdToNewId.contains(ref.cteId) && !newIdValues.contains(ref.cteId) =>
ref.copy(cteId = cteIdToNewId(ref.cteId))
}
} else {
normalized
}
}

private def canonicalizeCTE(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.catalyst.normalizer

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, LocalRelation, Union, WithCTE}
import org.apache.spark.sql.types.IntegerType

class NormalizeCTEIdsSuite extends SparkFunSuite {
test("SPARK-56739: orphan CTERelationRef outside WithCTE should be normalized") {
val attr = AttributeReference("id", IntegerType)()
val relation = LocalRelation(attr)

// CTE def with a high ID (simulates real usage where IDs are large)
val cteDef = CTERelationDef(relation, id = 100L)

// WithCTE with a ref inside
val innerRef = CTERelationRef(100L, _resolved = true, output = Seq(attr), isStreaming = false)
val withCTE = WithCTE(innerRef, Seq(cteDef))

// Orphan ref OUTSIDE the WithCTE (simulates post-InlineCTE/MergeSubplans state)
val orphanRef = CTERelationRef(100L, _resolved = true, output = Seq(attr), isStreaming = false)

// Plan: Union of WithCTE and orphan ref
val plan = Union(Seq(withCTE, orphanRef))

val normalized = NormalizeCTEIds(plan)

// Collect all CTERelationRef IDs in the normalized plan
val refIds = normalized.collect {
case ref: CTERelationRef => ref.cteId
}

// The ref inside WithCTE gets normalized to 0. The orphan should also be 0.
assert(refIds.nonEmpty, "Should have CTERelationRefs in the plan")
assert(refIds.forall(_ == 0L),
s"All CTERelationRef IDs should be normalized to 0 but got: $refIds")
}
}