Skip to content
Closed
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
2 changes: 1 addition & 1 deletion docs/sql-migration-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ license: |

- Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time.

- Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`.
- Since Spark 3.0, Spark throws an AnalysisException if name conflict is detected in the nested WITH clause by default. It forces the users to choose the specific substitution order they wanted, which is controlled by `spark.sql.legacy.ctePrecedence.enabled`. If set to false (which is recommended), inner CTE definitions take precedence over outer definitions. For example, set the config to `false`, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `2`, while setting it to `true`, the result is `1` which is the behavior in version 2.4 and earlier.

- Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, With}
import org.apache.spark.sql.catalyst.rules.Rule
Expand All @@ -28,10 +29,54 @@ import org.apache.spark.sql.internal.SQLConf.LEGACY_CTE_PRECEDENCE_ENABLED
*/
object CTESubstitution extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = {
if (SQLConf.get.getConf(LEGACY_CTE_PRECEDENCE_ENABLED)) {
val isLegacy = SQLConf.get.getConf(LEGACY_CTE_PRECEDENCE_ENABLED)
if (isLegacy.isEmpty) {
assertNoNameConflictsInCTE(plan, inTraverse = false)
traverseAndSubstituteCTE(plan, inTraverse = false)
} else if (isLegacy.get) {
legacyTraverseAndSubstituteCTE(plan)
} else {
traverseAndSubstituteCTE(plan, false)
traverseAndSubstituteCTE(plan, inTraverse = false)
}
}

/**
* Check the plan to be traversed has naming conflicts in nested CTE or not, traverse through
* child, innerChildren and subquery for the current plan.
*/
private def assertNoNameConflictsInCTE(
plan: LogicalPlan,
inTraverse: Boolean,
cteNames: Set[String] = Set.empty): Unit = {
plan.foreach {
case w @ With(child, relations) =>
val newNames = relations.map {
case (cteName, _) =>
if (cteNames.contains(cteName)) {
throw new AnalysisException(s"Name $cteName is ambiguous in nested CTE. " +
s"Please set ${LEGACY_CTE_PRECEDENCE_ENABLED.key} to false so that name defined " +
"in inner CTE takes precedence. See more details in SPARK-28228.")
} else {
cteName
}
}.toSet
child.transformExpressions {
case e: SubqueryExpression =>
assertNoNameConflictsInCTE(e.plan, inTraverse = true, cteNames ++ newNames)
e
}
w.innerChildren.foreach { p =>
assertNoNameConflictsInCTE(p, inTraverse = true, cteNames ++ newNames)
}

case other if inTraverse =>
other.transformExpressions {
case e: SubqueryExpression =>
assertNoNameConflictsInCTE(e.plan, inTraverse = true, cteNames)
e
}

case _ =>
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2077,9 +2077,11 @@ object SQLConf {

val LEGACY_CTE_PRECEDENCE_ENABLED = buildConf("spark.sql.legacy.ctePrecedence.enabled")
.internal()
.doc("When true, outer CTE definitions takes precedence over inner definitions.")
.doc("When true, outer CTE definitions takes precedence over inner definitions. If set to " +
"false, inner CTE definitions take precedence. The default value is empty, " +
"AnalysisException is thrown while name conflict is detected in nested CTE.")
.booleanConf
.createWithDefault(false)
.createOptional

val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC =
buildConf("spark.sql.legacy.arrayExistsFollowsThreeValuedLogic")
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
--SET spark.sql.legacy.ctePrecedence.enabled = false
--IMPORT cte.sql
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should do the same thing for cte-legacy.sql. This can be done in a followup.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy that, will do it later.

Loading