Skip to content

Commit b9783a9

Browse files
jiangxb1987cloud-fan
authored andcommitted
[SPARK-18389][SQL] Disallow cyclic view reference
## What changes were proposed in this pull request? Disallow cyclic view references, a cyclic view reference may be created by the following queries: ``` CREATE VIEW testView AS SELECT id FROM tbl CREATE VIEW testView2 AS SELECT id FROM testView ALTER VIEW testView AS SELECT * FROM testView2 ``` In the above example, a reference cycle (testView -> testView2 -> testView) exsits. We disallow cyclic view references by checking that in ALTER VIEW command, when the `analyzedPlan` contains the same `View` node with the altered view, we should prevent the behavior and throw an AnalysisException. ## How was this patch tested? Test by `SQLViewSuite.test("correctly handle a cyclic view reference")`. Author: jiangxingbo <[email protected]> Closes #17152 from jiangxb1987/cyclic-view.
1 parent c96d14a commit b9783a9

File tree

2 files changed

+90
-6
lines changed

2 files changed

+90
-6
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala

Lines changed: 59 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,9 +23,9 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
2323
import org.apache.spark.sql.catalyst.TableIdentifier
2424
import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, UnresolvedRelation}
2525
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType}
26-
import org.apache.spark.sql.catalyst.expressions.Alias
26+
import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression}
2727
import org.apache.spark.sql.catalyst.plans.QueryPlan
28-
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
28+
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View}
2929
import org.apache.spark.sql.types.MetadataBuilder
3030

3131

@@ -154,6 +154,10 @@ case class CreateViewCommand(
154154
} else if (tableMetadata.tableType != CatalogTableType.VIEW) {
155155
throw new AnalysisException(s"$name is not a view")
156156
} else if (replace) {
157+
// Detect cyclic view reference on CREATE OR REPLACE VIEW.
158+
val viewIdent = tableMetadata.identifier
159+
checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent)
160+
157161
// Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...`
158162
catalog.alterTable(prepareTable(sparkSession, analyzedPlan))
159163
} else {
@@ -283,6 +287,10 @@ case class AlterViewAsCommand(
283287
throw new AnalysisException(s"${viewMeta.identifier} is not a view.")
284288
}
285289

290+
// Detect cyclic view reference on ALTER VIEW.
291+
val viewIdent = viewMeta.identifier
292+
checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent)
293+
286294
val newProperties = generateViewProperties(viewMeta.properties, session, analyzedPlan)
287295

288296
val updatedViewMeta = viewMeta.copy(
@@ -358,4 +366,53 @@ object ViewHelper {
358366
generateViewDefaultDatabase(viewDefaultDatabase) ++
359367
generateQueryColumnNames(queryOutput)
360368
}
369+
370+
/**
371+
* Recursively search the logical plan to detect cyclic view references, throw an
372+
* AnalysisException if cycle detected.
373+
*
374+
* A cyclic view reference is a cycle of reference dependencies, for example, if the following
375+
* statements are executed:
376+
* CREATE VIEW testView AS SELECT id FROM tbl
377+
* CREATE VIEW testView2 AS SELECT id FROM testView
378+
* ALTER VIEW testView AS SELECT * FROM testView2
379+
* The view `testView` references `testView2`, and `testView2` also references `testView`,
380+
* therefore a reference cycle (testView -> testView2 -> testView) exists.
381+
*
382+
* @param plan the logical plan we detect cyclic view references from.
383+
* @param path the path between the altered view and current node.
384+
* @param viewIdent the table identifier of the altered view, we compare two views by the
385+
* `desc.identifier`.
386+
*/
387+
def checkCyclicViewReference(
388+
plan: LogicalPlan,
389+
path: Seq[TableIdentifier],
390+
viewIdent: TableIdentifier): Unit = {
391+
plan match {
392+
case v: View =>
393+
val ident = v.desc.identifier
394+
val newPath = path :+ ident
395+
// If the table identifier equals to the `viewIdent`, current view node is the same with
396+
// the altered view. We detect a view reference cycle, should throw an AnalysisException.
397+
if (ident == viewIdent) {
398+
throw new AnalysisException(s"Recursive view $viewIdent detected " +
399+
s"(cycle: ${newPath.mkString(" -> ")})")
400+
} else {
401+
v.children.foreach { child =>
402+
checkCyclicViewReference(child, newPath, viewIdent)
403+
}
404+
}
405+
case _ =>
406+
plan.children.foreach(child => checkCyclicViewReference(child, path, viewIdent))
407+
}
408+
409+
// Detect cyclic references from subqueries.
410+
plan.expressions.foreach { expr =>
411+
expr match {
412+
case s: SubqueryExpression =>
413+
checkCyclicViewReference(s.plan, path, viewIdent)
414+
case _ => // Do nothing.
415+
}
416+
}
417+
}
361418
}

sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala

Lines changed: 31 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -609,12 +609,39 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
609609
}
610610
}
611611

612-
// TODO: Check for cyclic view references on ALTER VIEW.
613-
ignore("correctly handle a cyclic view reference") {
614-
withView("view1", "view2") {
612+
test("correctly handle a cyclic view reference") {
613+
withView("view1", "view2", "view3") {
615614
sql("CREATE VIEW view1 AS SELECT * FROM jt")
616615
sql("CREATE VIEW view2 AS SELECT * FROM view1")
617-
intercept[AnalysisException](sql("ALTER VIEW view1 AS SELECT * FROM view2"))
616+
sql("CREATE VIEW view3 AS SELECT * FROM view2")
617+
618+
// Detect cyclic view reference on ALTER VIEW.
619+
val e1 = intercept[AnalysisException] {
620+
sql("ALTER VIEW view1 AS SELECT * FROM view2")
621+
}.getMessage
622+
assert(e1.contains("Recursive view `default`.`view1` detected (cycle: `default`.`view1` " +
623+
"-> `default`.`view2` -> `default`.`view1`)"))
624+
625+
// Detect the most left cycle when there exists multiple cyclic view references.
626+
val e2 = intercept[AnalysisException] {
627+
sql("ALTER VIEW view1 AS SELECT * FROM view3 JOIN view2")
628+
}.getMessage
629+
assert(e2.contains("Recursive view `default`.`view1` detected (cycle: `default`.`view1` " +
630+
"-> `default`.`view3` -> `default`.`view2` -> `default`.`view1`)"))
631+
632+
// Detect cyclic view reference on CREATE OR REPLACE VIEW.
633+
val e3 = intercept[AnalysisException] {
634+
sql("CREATE OR REPLACE VIEW view1 AS SELECT * FROM view2")
635+
}.getMessage
636+
assert(e3.contains("Recursive view `default`.`view1` detected (cycle: `default`.`view1` " +
637+
"-> `default`.`view2` -> `default`.`view1`)"))
638+
639+
// Detect cyclic view reference from subqueries.
640+
val e4 = intercept[AnalysisException] {
641+
sql("ALTER VIEW view1 AS SELECT * FROM jt WHERE EXISTS (SELECT 1 FROM view2)")
642+
}.getMessage
643+
assert(e4.contains("Recursive view `default`.`view1` detected (cycle: `default`.`view1` " +
644+
"-> `default`.`view2` -> `default`.`view1`)"))
618645
}
619646
}
620647
}

0 commit comments

Comments
 (0)