From db4dfaf4f455c2abda7d103e16b9a9e0a207f570 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 18 Jan 2021 18:07:17 -0800 Subject: [PATCH 01/30] initial commit --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 9 +++- .../plans/logical/basicLogicalOperators.scala | 3 ++ .../catalyst/plans/logical/statements.scala | 5 +- .../spark/sql/execution/command/views.scala | 20 ++++--- .../sql-tests/results/explain-aqe.sql.out | 28 +++++++--- .../sql-tests/results/explain.sql.out | 28 +++++++--- .../sql/connector/DataSourceV2SQLSuite.scala | 52 +++++++++++++++++++ 8 files changed, 124 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e33663a3fc062..0e76ffda52a66 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1086,7 +1086,7 @@ class Analyzer(override val catalogManager: CatalogManager) } view.copy(child = newChild) case p @ SubqueryAlias(_, view: View) => - p.copy(child = resolveViews(view)) + p.makeCopy(Array(p.identifier, resolveViews(view))) case _ => plan } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 0428d12b7ced8..2496357387c3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, ImplicitCastInputTypes} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias.SUBQUERY_TYPE_TAG import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils} import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.errors.QueryCompilationErrors @@ -854,7 +855,9 @@ class SessionCatalog( def lookupTempView(table: String): Option[SubqueryAlias] = { val formattedTable = formatTableName(table) getTempView(formattedTable).map { view => - SubqueryAlias(formattedTable, view) + val s = SubqueryAlias(formattedTable, view) + s.setTagValue(SUBQUERY_TYPE_TAG, "tempView") + s } } @@ -863,7 +866,9 @@ class SessionCatalog( if (formattedDB == globalTempViewManager.database) { val formattedTable = formatTableName(table) getGlobalTempView(formattedTable).map { view => - SubqueryAlias(formattedTable, formattedDB, view) + val s = SubqueryAlias(formattedTable, formattedDB, view) + s.setTagValue(SUBQUERY_TYPE_TAG, "tempView") + s } } else { None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 80883e1cd5e43..88ed55957dea9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -954,6 +955,8 @@ case class SubqueryAlias( } object SubqueryAlias { + val SUBQUERY_TYPE_TAG = TreeNodeTag[String]("subQueryType") + def apply( identifier: String, child: LogicalPlan): SubqueryAlias = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index f6d141ded384a..854b1c9257922 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -180,7 +180,10 @@ case class CreateViewStatement( child: LogicalPlan, allowExisting: Boolean, replace: Boolean, - viewType: ViewType) extends ParsedStatement + viewType: ViewType) extends ParsedStatement { + + override def children: Seq[LogicalPlan] = Seq(child) +} /** * A REPLACE TABLE command, as parsed from SQL. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index eed4d92706bcf..662e8565d53cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -23,12 +23,13 @@ import org.json4s.JsonAST.{JArray, JString} import org.json4s.jackson.JsonMethods._ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, UnresolvedFunction, UnresolvedRelation, ViewType} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, UnresolvedRelation, ViewType} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, SessionCatalog, TemporaryViewRelation} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, PythonUDF, ScalaUDF, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias.SUBQUERY_TYPE_TAG import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} @@ -558,6 +559,8 @@ object ViewHelper { catalog: SessionCatalog, child: LogicalPlan): (Seq[Seq[String]], Seq[String]) = { def collectTempViews(child: LogicalPlan): Seq[Seq[String]] = { child.collect { + case s: SubqueryAlias if s.getTagValue(SUBQUERY_TYPE_TAG).exists(_ == "tempView") => + Seq(s.identifier.qualifier :+ s.identifier.name) case UnresolvedRelation(nameParts, _, _) if catalog.isTempView(nameParts) => Seq(nameParts) case plan if !plan.resolved => plan.expressions.flatMap(_.collect { @@ -568,10 +571,13 @@ object ViewHelper { def collectTempFunctions(child: LogicalPlan): Seq[String] = { child.collect { - case plan if !plan.resolved => plan.expressions.flatMap(_.collect { + case plan if plan.resolved => plan.expressions.flatMap(_.collect { case e: SubqueryExpression => collectTempFunctions(e.plan) - case e: UnresolvedFunction if catalog.isTemporaryFunction(e.name) => - Seq(e.name.funcName) + case e: PythonUDF if catalog.isTemporaryFunction(FunctionIdentifier(e.name)) => + Seq(e.name) + case e: ScalaUDF + if e.udfName.exists(name => catalog.isTemporaryFunction(FunctionIdentifier(name))) => + Seq(e.udfName.get) }).flatten }.flatten.distinct } diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index 61bf346d51d6f..eb4384f7b526e 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -904,8 +904,9 @@ struct == Physical Plan == Execute CreateViewCommand (1) +- CreateViewCommand (2) - +- Project (4) - +- UnresolvedRelation (3) + +- Project (5) + +- SubqueryAlias (4) + +- LogicalRelation (3) (1) Execute CreateViewCommand @@ -914,10 +915,25 @@ Output: [] (2) CreateViewCommand Arguments: `default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView -(3) UnresolvedRelation -Arguments: [explain_temp1], [], false - -(4) Project +(3) LogicalRelation +Arguments: parquet, [key#x, val#x], CatalogTable( +Database: default +Table: explain_temp1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: MANAGED +Provider: PARQUET +Location [not included in comparison]/{warehouse_dir}/explain_temp1 +Schema: root +-- key: integer (nullable = true) +-- val: integer (nullable = true) +), false + +(4) SubqueryAlias +Arguments: spark_catalog.default.explain_temp1 + +(5) Project Arguments: ['key, 'val] diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index 59116f5f5b6c6..f92c0c6df1f09 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -849,8 +849,9 @@ struct == Physical Plan == Execute CreateViewCommand (1) +- CreateViewCommand (2) - +- Project (4) - +- UnresolvedRelation (3) + +- Project (5) + +- SubqueryAlias (4) + +- LogicalRelation (3) (1) Execute CreateViewCommand @@ -859,10 +860,25 @@ Output: [] (2) CreateViewCommand Arguments: `default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView -(3) UnresolvedRelation -Arguments: [explain_temp1], [], false - -(4) Project +(3) LogicalRelation +Arguments: parquet, [key#x, val#x], CatalogTable( +Database: default +Table: explain_temp1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: MANAGED +Provider: PARQUET +Location [not included in comparison]/{warehouse_dir}/explain_temp1 +Schema: root +-- key: integer (nullable = true) +-- val: integer (nullable = true) +), false + +(4) SubqueryAlias +Arguments: spark_catalog.default.explain_temp1 + +(5) Project Arguments: ['key, 'val] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 279947195c418..f83a58ce4d710 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -26,11 +26,13 @@ import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.internal.connector.SimpleTableProvider @@ -2277,6 +2279,56 @@ class DataSourceV2SQLSuite } } + test("spark-34152") { + // unset this config to use the default v2 session catalog. + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + + val udf = new SparkUserDefinedFunction( + (input: Any) => if (input == null) { + null + } else { + input.toString + }, + StringType, + inputEncoders = Seq.fill(1)(None), + name = Some("udf")) { + + override def apply(exprs: Column*): Column = { + assert(exprs.length == 1, "Defined UDF only has one column") + val expr = exprs.head.expr + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Column(Cast(createScalaUDF(Cast(expr, StringType) :: Nil), expr.dataType)) + } + } + + spark.udf.register("udf", udf) + + // withSQLConf("spark.sql.legacy.storeAnalyzedPlanForView" -> "true") { + sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a)") + sql("CREATE TEMPORARY VIEW ta AS SELECT udf(a) AS a FROM t1") + sql("SELECT * from ta") + // } + + +// sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a)") +// sql("CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a)") +// sql("CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a)") +// sql("CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a)") +// +// sql("CREATE TEMPORARY VIEW ta AS " + +// "SELECT udf(a) AS a, udf('a') AS tag FROM t1 " + +// "UNION ALL " + +// "SELECT udf(a) AS a, udf('b') AS tag FROM t2") +// +// sql("CREATE TEMPORARY VIEW tb AS " + +// "SELECT udf(a) AS a, udf('a') AS tag FROM t3 " + +// "UNION ALL " + +// "SELECT udf(a) AS a, udf('b') AS tag FROM t4") +// +// sql("SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag") + } + test("SPARK-30284: CREATE VIEW should track the current catalog and namespace") { // unset this config to use the default v2 session catalog. spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) From 71c01e85c7eb9a530607078730dd5cb8bc533be6 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 20 Jan 2021 20:52:54 -0800 Subject: [PATCH 02/30] fix tests --- .../sql/catalyst/catalog/SessionCatalog.scala | 11 ++-- .../catalyst/plans/logical/statements.scala | 2 +- .../apache/spark/sql/UDFRegistration.scala | 2 +- .../analysis/ResolveSessionCatalog.scala | 2 +- .../spark/sql/execution/aggregate/udaf.scala | 3 +- .../spark/sql/execution/command/views.scala | 19 ++++--- .../sql/connector/DataSourceV2SQLSuite.scala | 54 +------------------ 7 files changed, 25 insertions(+), 68 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 2496357387c3e..253becb6749c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1368,9 +1368,14 @@ class SessionCatalog( Utils.classForName("org.apache.spark.sql.expressions.UserDefinedAggregateFunction") if (clsForUDAF.isAssignableFrom(clazz)) { val cls = Utils.classForName("org.apache.spark.sql.execution.aggregate.ScalaUDAF") - val e = cls.getConstructor(classOf[Seq[Expression]], clsForUDAF, classOf[Int], classOf[Int]) - .newInstance(input, - clazz.getConstructor().newInstance().asInstanceOf[Object], Int.box(1), Int.box(1)) + val e = cls.getConstructor( + classOf[Seq[Expression]], clsForUDAF, classOf[Int], classOf[Int], classOf[Option[String]]) + .newInstance( + input, + clazz.getConstructor().newInstance().asInstanceOf[Object], + Int.box(1), + Int.box(1), + Some(name)) .asInstanceOf[ImplicitCastInputTypes] // Check input argument size diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 854b1c9257922..058372fb7c1ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -182,7 +182,7 @@ case class CreateViewStatement( replace: Boolean, viewType: ViewType) extends ParsedStatement { - override def children: Seq[LogicalPlan] = Seq(child) + override def children: Seq[LogicalPlan] = Seq(child) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 7567ed63359a9..73131438279fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -82,7 +82,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends @deprecated("Aggregator[IN, BUF, OUT] should now be registered as a UDF" + " via the functions.udaf(agg) method.", "3.0.0") def register(name: String, udaf: UserDefinedAggregateFunction): UserDefinedAggregateFunction = { - def builder(children: Seq[Expression]) = ScalaUDAF(children, udaf) + def builder(children: Seq[Expression]) = ScalaUDAF(children, udaf, name = Some(name)) functionRegistry.createOrReplaceTempFunction(name, builder) udaf } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 16cd2068fce52..f69ae8aa21f6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -476,7 +476,7 @@ class ResolveSessionCatalog( case CreateViewStatement( tbl, userSpecifiedColumns, comment, properties, - originalText, child, allowExisting, replace, viewType) => + originalText, child, allowExisting, replace, viewType) if child.resolved => val v1TableName = if (viewType != PersistedView) { // temp view doesn't belong to any catalog and we shouldn't resolve catalog in the name. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index 41e247a02759b..043a8d0b648a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -325,7 +325,8 @@ case class ScalaUDAF( children: Seq[Expression], udaf: UserDefinedAggregateFunction, mutableAggBufferOffset: Int = 0, - inputAggBufferOffset: Int = 0) + inputAggBufferOffset: Int = 0, + name: Option[String] = None) extends ImperativeAggregate with NonSQLExpression with Logging diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 662e8565d53cd..6d88ee1a87814 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -24,7 +24,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, UnresolvedRelation, ViewType} +import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, ViewType} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, SessionCatalog, TemporaryViewRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, PythonUDF, ScalaUDF, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Subque import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias.SUBQUERY_TYPE_TAG import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper +import org.apache.spark.sql.execution.aggregate.ScalaUDAF import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType, StructType} import org.apache.spark.sql.util.SchemaUtils @@ -559,24 +560,26 @@ object ViewHelper { catalog: SessionCatalog, child: LogicalPlan): (Seq[Seq[String]], Seq[String]) = { def collectTempViews(child: LogicalPlan): Seq[Seq[String]] = { child.collect { + case s @ SubqueryAlias(_, view: View) if view.isTempView => + Seq(s.identifier.qualifier :+ s.identifier.name) case s: SubqueryAlias if s.getTagValue(SUBQUERY_TYPE_TAG).exists(_ == "tempView") => Seq(s.identifier.qualifier :+ s.identifier.name) - case UnresolvedRelation(nameParts, _, _) if catalog.isTempView(nameParts) => - Seq(nameParts) - case plan if !plan.resolved => plan.expressions.flatMap(_.collect { + case plan if plan.resolved => plan.expressions.flatMap(_.collect { case e: SubqueryExpression => collectTempViews(e.plan) }).flatten }.flatten.distinct } def collectTempFunctions(child: LogicalPlan): Seq[String] = { + def isTemporaryFunction(name: String): Boolean = { + catalog.isTemporaryFunction(FunctionIdentifier(name)) + } child.collect { case plan if plan.resolved => plan.expressions.flatMap(_.collect { case e: SubqueryExpression => collectTempFunctions(e.plan) - case e: PythonUDF if catalog.isTemporaryFunction(FunctionIdentifier(e.name)) => - Seq(e.name) - case e: ScalaUDF - if e.udfName.exists(name => catalog.isTemporaryFunction(FunctionIdentifier(name))) => + case e: PythonUDF if isTemporaryFunction(e.name) => Seq(e.name) + case e: ScalaUDAF if e.name.exists(name => isTemporaryFunction(name)) => Seq(e.name.get) + case e: ScalaUDF if e.udfName.exists(name => isTemporaryFunction(name)) => Seq(e.udfName.get) }).flatten }.flatten.distinct diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index f83a58ce4d710..288c096a12448 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -26,13 +26,11 @@ import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, TableAlreadyExistsException} -import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership import org.apache.spark.sql.execution.columnar.InMemoryRelation -import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.internal.connector.SimpleTableProvider @@ -2031,7 +2029,7 @@ class DataSourceV2SQLSuite test("CREATE VIEW") { val v = "testcat.ns1.ns2.v" val e = intercept[AnalysisException] { - sql(s"CREATE VIEW $v AS SELECT * FROM tab1") + sql(s"CREATE VIEW $v AS SELECT 1") } assert(e.message.contains("CREATE VIEW is only supported with v1 tables")) } @@ -2279,56 +2277,6 @@ class DataSourceV2SQLSuite } } - test("spark-34152") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) - - val udf = new SparkUserDefinedFunction( - (input: Any) => if (input == null) { - null - } else { - input.toString - }, - StringType, - inputEncoders = Seq.fill(1)(None), - name = Some("udf")) { - - override def apply(exprs: Column*): Column = { - assert(exprs.length == 1, "Defined UDF only has one column") - val expr = exprs.head.expr - assert(expr.resolved, "column should be resolved to use the same type " + - "as input. Try df(name) or df.col(name)") - Column(Cast(createScalaUDF(Cast(expr, StringType) :: Nil), expr.dataType)) - } - } - - spark.udf.register("udf", udf) - - // withSQLConf("spark.sql.legacy.storeAnalyzedPlanForView" -> "true") { - sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a)") - sql("CREATE TEMPORARY VIEW ta AS SELECT udf(a) AS a FROM t1") - sql("SELECT * from ta") - // } - - -// sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a)") -// sql("CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a)") -// sql("CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a)") -// sql("CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a)") -// -// sql("CREATE TEMPORARY VIEW ta AS " + -// "SELECT udf(a) AS a, udf('a') AS tag FROM t1 " + -// "UNION ALL " + -// "SELECT udf(a) AS a, udf('b') AS tag FROM t2") -// -// sql("CREATE TEMPORARY VIEW tb AS " + -// "SELECT udf(a) AS a, udf('a') AS tag FROM t3 " + -// "UNION ALL " + -// "SELECT udf(a) AS a, udf('b') AS tag FROM t4") -// -// sql("SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag") - } - test("SPARK-30284: CREATE VIEW should track the current catalog and namespace") { // unset this config to use the default v2 session catalog. spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) From 8dc19613500b6301d7c666680fbb153ca41fb04b Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 22 Jan 2021 21:11:03 -0800 Subject: [PATCH 03/30] wrap with View --- .../sql/catalyst/analysis/Analyzer.scala | 7 +++- .../sql/catalyst/catalog/SessionCatalog.scala | 9 +---- .../plans/logical/basicLogicalOperators.scala | 3 -- .../spark/sql/execution/command/views.scala | 39 +++++++++---------- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- 5 files changed, 27 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0e76ffda52a66..4f409756fdd18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1086,7 +1086,7 @@ class Analyzer(override val catalogManager: CatalogManager) } view.copy(child = newChild) case p @ SubqueryAlias(_, view: View) => - p.makeCopy(Array(p.identifier, resolveViews(view))) + p.copy(child = resolveViews(view)) case _ => plan } @@ -1099,6 +1099,11 @@ class Analyzer(override val catalogManager: CatalogManager) } EliminateSubqueryAliases(relation) match { + case v: View if v.isTempView && v.child.isInstanceOf[LeafNode] => + // Inserting into a file-based temporary view is allowed. + // (e.g., spark.read.parquet("path").createOrReplaceTempView("t"). + // The check on the rdd-based relation is done in PreWriteCheck. + i.copy(table = v.child) case v: View => throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.identifier, table) case other => i.copy(table = other) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 253becb6749c1..7054588427e70 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, ImplicitCastInputTypes} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} -import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias.SUBQUERY_TYPE_TAG import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils} import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.errors.QueryCompilationErrors @@ -855,9 +854,7 @@ class SessionCatalog( def lookupTempView(table: String): Option[SubqueryAlias] = { val formattedTable = formatTableName(table) getTempView(formattedTable).map { view => - val s = SubqueryAlias(formattedTable, view) - s.setTagValue(SUBQUERY_TYPE_TAG, "tempView") - s + SubqueryAlias(formattedTable, view) } } @@ -866,9 +863,7 @@ class SessionCatalog( if (formattedDB == globalTempViewManager.database) { val formattedTable = formatTableName(table) getGlobalTempView(formattedTable).map { view => - val s = SubqueryAlias(formattedTable, formattedDB, view) - s.setTagValue(SUBQUERY_TYPE_TAG, "tempView") - s + SubqueryAlias(formattedTable, formattedDB, view) } } else { None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 88ed55957dea9..80883e1cd5e43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} -import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -955,8 +954,6 @@ case class SubqueryAlias( } object SubqueryAlias { - val SUBQUERY_TYPE_TAG = TreeNodeTag[String]("subQueryType") - def apply( identifier: String, child: LogicalPlan): SubqueryAlias = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 6d88ee1a87814..2eb2262b52a95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, PythonUDF, ScalaUDF, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View} -import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias.SUBQUERY_TYPE_TAG import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.execution.aggregate.ScalaUDAF @@ -123,18 +122,18 @@ case class CreateViewCommand( CommandUtils.uncacheTableOrView(sparkSession, name.quotedString) } val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) + val catalogTable = prepareTemporaryView( + name, + sparkSession, + analyzedPlan, + aliasedPlan.schema, + originalText, + child) // If there is no sql text (e.g. from Dataset API), we will always store the analyzed plan val tableDefinition = if (!conf.storeAnalyzedPlanForView && originalText.nonEmpty) { - TemporaryViewRelation( - prepareTemporaryView( - name, - sparkSession, - analyzedPlan, - aliasedPlan.schema, - originalText, - child)) + TemporaryViewRelation(catalogTable) } else { - aliasedPlan + View(catalogTable, isTemporary, catalogTable.schema.toAttributes, child) } catalog.createTempView(name.table, tableDefinition, overrideIfExists = replace) } else if (viewType == GlobalTempView) { @@ -147,17 +146,17 @@ case class CreateViewCommand( CommandUtils.uncacheTableOrView(sparkSession, viewIdent.quotedString) } val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) + val catalogTable = prepareTemporaryView( + viewIdent, + sparkSession, + analyzedPlan, + aliasedPlan.schema, + originalText, + child) val tableDefinition = if (!conf.storeAnalyzedPlanForView && originalText.nonEmpty) { - TemporaryViewRelation( - prepareTemporaryView( - viewIdent, - sparkSession, - analyzedPlan, - aliasedPlan.schema, - originalText, - child)) + TemporaryViewRelation(catalogTable) } else { - aliasedPlan + View(catalogTable, isTemporary, catalogTable.schema.toAttributes, child) } catalog.createGlobalTempView(name.table, tableDefinition, overrideIfExists = replace) } else if (catalog.tableExists(name)) { @@ -562,8 +561,6 @@ object ViewHelper { child.collect { case s @ SubqueryAlias(_, view: View) if view.isTempView => Seq(s.identifier.qualifier :+ s.identifier.name) - case s: SubqueryAlias if s.getTagValue(SUBQUERY_TYPE_TAG).exists(_ == "tempView") => - Seq(s.identifier.qualifier :+ s.identifier.name) case plan if plan.resolved => plan.expressions.flatMap(_.collect { case e: SubqueryExpression => collectTempViews(e.plan) }).flatten diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index f36038222193b..7ee58f6824345 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1548,7 +1548,7 @@ class DataFrameSuite extends QueryTest val e2 = intercept[AnalysisException] { insertion.write.insertInto("indirect_ds") } - assert(e2.getMessage.contains("Inserting into an RDD-based table is not allowed.")) + assert(e2.getMessage.contains("Inserting into a view is not allowed. View: `indirect_ds`")) // error case: insert into an OneRowRelation Dataset.ofRows(spark, OneRowRelation()).createOrReplaceTempView("one_row") From 4b3f184ec49b14a6c445ea57d6f562564ef3474d Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 25 Jan 2021 14:02:54 -0800 Subject: [PATCH 04/30] trying with View and optional CatalogTable --- .../sql/catalyst/analysis/Analyzer.scala | 36 ++++++++++++---- .../sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../spark/sql/catalyst/analysis/view.scala | 16 ++++++- .../sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../plans/logical/basicLogicalOperators.scala | 6 ++- .../sql/catalyst/analysis/AnalysisSuite.scala | 4 +- .../catalog/SessionCatalogSuite.scala | 10 ++++- .../spark/sql/execution/CacheManager.scala | 10 +++-- .../spark/sql/execution/command/views.scala | 42 ++++++++++--------- .../spark/sql/internal/CatalogImpl.scala | 2 +- 11 files changed, 88 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d97e067425bbb..6971b745adf18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -884,10 +884,14 @@ class Analyzer(override val catalogManager: CatalogManager) case write: V2WriteCommand => write.table match { case UnresolvedRelation(ident, _, false) => - lookupTempView(ident).map(EliminateSubqueryAliases(_)).map { - case r: DataSourceV2Relation => write.withNewTable(r) - case _ => throw QueryCompilationErrors.writeIntoTempViewNotAllowedError(ident.quoted) - }.getOrElse(write) + lookupTempView(ident) + .map(EliminateSubqueryAliases(_)) + .map(EliminateDataFrameTempViews(_)) + .map { + case r: DataSourceV2Relation => write.withNewTable(r) + case _ => + throw QueryCompilationErrors.writeIntoTempViewNotAllowedError(ident.quoted) + }.getOrElse(write) case _ => write } case u @ UnresolvedTable(ident, cmd) => @@ -1071,7 +1075,7 @@ class Analyzer(override val catalogManager: CatalogManager) // The view's child should be a logical plan parsed from the `desc.viewText`, the variable // `viewText` should be defined, or else we throw an error on the generation of the View // operator. - case view @ View(desc, isTempView, _, child) if !child.resolved => + case view @ View(Some(desc), isTempView, _, child) if !child.resolved => // Resolve all the UnresolvedRelations and Views in the child. val newChild = AnalysisContext.withAnalysisContext(desc) { val nestedViewDepth = AnalysisContext.get.nestedViewDepth @@ -1085,6 +1089,10 @@ class Analyzer(override val catalogManager: CatalogManager) } } view.copy(child = newChild) + // If view.desc is None, view must be storing a dataframe temp view. + case view @ View(None, isTempView, _, child) => + assert(isTempView && child.resolved) + view case p @ SubqueryAlias(_, view: View) => p.copy(child = resolveViews(view)) case _ => plan @@ -1104,8 +1112,8 @@ class Analyzer(override val catalogManager: CatalogManager) // (e.g., spark.read.parquet("path").createOrReplaceTempView("t"). // The check on the rdd-based relation is done in PreWriteCheck. i.copy(table = v.child) - case v: View => - throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.identifier, table) + case v: View if v.desc.isDefined => + throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.get.identifier, table) case other => i.copy(table = other) } @@ -1130,8 +1138,9 @@ class Analyzer(override val catalogManager: CatalogManager) lookupRelation(u.multipartIdentifier, u.options, false) .map(EliminateSubqueryAliases(_)) .map { - case v: View => throw QueryCompilationErrors.writeIntoViewNotAllowedError( - v.desc.identifier, write) + case v: View if v.desc.isDefined => + throw QueryCompilationErrors.writeIntoViewNotAllowedError( + v.desc.get.identifier, write) case u: UnresolvedCatalogRelation => throw QueryCompilationErrors.writeIntoV1TableNotAllowedError( u.tableMeta.identifier, write) @@ -3671,6 +3680,15 @@ object EliminateSubqueryAliases extends Rule[LogicalPlan] { } } +/** + * Removes [[View]] operators from the plan if they are temp views created by DataFrame API. + */ +object EliminateDataFrameTempViews extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case v: View if v.desc.isEmpty => v.child + } +} + /** * Removes [[Union]] operators from the plan if it just has one child. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 8ec9583125fd3..5232de8c4b75d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -426,7 +426,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { // output, nor with the query column names, throw an AnalysisException. // If the view's child output can't up cast to the view output, // throw an AnalysisException, too. - case v @ View(desc, _, output, child) if child.resolved && !v.sameOutput(child) => + case v @ View(Some(desc), _, output, child) if child.resolved && !v.sameOutput(child) => val queryColumnNames = desc.viewQueryColumnNames val queryOutput = if (queryColumnNames.nonEmpty) { if (output.length != queryColumnNames.length) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala index dfadf0a539948..f31f34a543c1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala @@ -56,7 +56,7 @@ object EliminateView extends Rule[LogicalPlan] with CastSupport { override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { // The child has the different output attributes with the View operator. Adds a Project over // the child of the view. - case v @ View(desc, _, output, child) if child.resolved && !v.sameOutput(child) => + case v @ View(Some(desc), _, output, child) if child.resolved && !v.sameOutput(child) => val resolver = conf.resolver val queryColumnNames = desc.viewQueryColumnNames val queryOutput = if (queryColumnNames.nonEmpty) { @@ -81,9 +81,21 @@ object EliminateView extends Rule[LogicalPlan] with CastSupport { } Project(newOutput, child) + case v @ View(None, _, output, child) if !v.sameOutput(child) => + val queryOutput = child.output + // Map the attributes in the query output to the attributes in the view output by index. + val newOutput = output.zip(queryOutput).map { + case (attr, originAttr) if !attr.semanticEquals(originAttr) => + // `CheckAnalysis` already guarantees that the cast is a up-cast for sure. + Alias(cast(originAttr, attr.dataType), attr.name)(exprId = attr.exprId, + qualifier = attr.qualifier, explicitMetadata = Some(attr.metadata)) + case (_, originAttr) => originAttr + } + Project(newOutput, child) + // The child should have the same output attributes with the View operator, so we simply // remove the View operator. - case View(_, _, _, child) => + case v @ View(_, _, _, child) => child } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 0ae9cae8af779..f95150ec37b69 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -852,7 +852,7 @@ class SessionCatalog( parser.parsePlan(viewText) } View( - desc = metadata, + desc = Some(metadata), isTempView = isTempView, output = metadata.schema.toAttributes, child = viewPlan) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 28dfc0958ab6f..ff38aa066f1c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -143,6 +143,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateResolvedHint, EliminateSubqueryAliases, EliminateView, + EliminateDataFrameTempViews, ReplaceExpressions, RewriteNonCorrelatedExists, ComputeCurrentTime, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 80883e1cd5e43..7d35380b3f915 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -456,10 +456,11 @@ case class InsertIntoDir( * `CatalogTable.viewText`, should throw an error if the `viewText` is not defined. */ case class View( - desc: CatalogTable, + desc: Option[CatalogTable], isTempView: Boolean, output: Seq[Attribute], child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation { + require(desc.isDefined || isTempView) override def producedAttributes: AttributeSet = outputSet @@ -470,7 +471,8 @@ case class View( override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) override def simpleString(maxFields: Int): String = { - s"View (${desc.identifier}, ${output.mkString("[", ",", "]")})" + val viewIdent = desc.map{ d => s"${d.identifier}, "}.getOrElse("") + s"View ($viewIdent${output.mkString("[", ",", "]")})" } override def doCanonicalize(): LogicalPlan = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index f66871ee75ecc..c626f25e67f26 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -660,11 +660,11 @@ class AnalysisSuite extends AnalysisTest with Matchers { val batches = Batch("View", Once, EliminateView) :: Nil } val relation = LocalRelation(Symbol("a").int.notNull, Symbol("b").string) - val view = View(CatalogTable( + val view = View(Some(CatalogTable( identifier = TableIdentifier("v1"), tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, - schema = StructType(Seq(StructField("a", IntegerType), StructField("b", StringType)))), + schema = StructType(Seq(StructField("a", IntegerType), StructField("b", StringType))))), isTempView = false, output = Seq(Symbol("a").int, Symbol("b").string), child = relation) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 98f9ce6fe9dbb..7a53dbb17de3f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -646,7 +646,10 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { // Look up a view. catalog.setCurrentDatabase("default") - val view = View(desc = metadata, isTempView = false, output = metadata.schema.toAttributes, + val view = View( + desc = Some(metadata), + isTempView = false, + output = metadata.schema.toAttributes, child = CatalystSqlParser.parsePlan(metadata.viewText.get)) comparePlans(catalog.lookupRelation(TableIdentifier("view1", Some("db3"))), SubqueryAlias(Seq(CatalogManager.SESSION_CATALOG_NAME, "db3", "view1"), view)) @@ -666,7 +669,10 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { assert(metadata.viewText.isDefined) assert(metadata.viewCatalogAndNamespace == Seq(CatalogManager.SESSION_CATALOG_NAME, "db2")) - val view = View(desc = metadata, isTempView = false, output = metadata.schema.toAttributes, + val view = View( + desc = Some(metadata), + isTempView = false, + output = metadata.schema.toAttributes, child = CatalystSqlParser.parsePlan(metadata.viewText.get)) comparePlans(catalog.lookupRelation(TableIdentifier("view2", Some("db3"))), SubqueryAlias(Seq(CatalogManager.SESSION_CATALOG_NAME, "db3", "view2"), view)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 0c6f22dea7b45..bb20c03c9893a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -159,11 +159,15 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { plan: LogicalPlan, cascade: Boolean, blocking: Boolean = false): Unit = { + val qe = spark.sessionState.executePlan(plan) + qe.assertAnalyzed() + val analyzedPlan = qe.analyzed + val shouldRemove: LogicalPlan => Boolean = if (cascade) { - _.find(_.sameResult(plan)).isDefined + _.find(_.sameResult(analyzedPlan)).isDefined } else { - _.sameResult(plan) + _.sameResult(analyzedPlan) } val plansToUncache = cachedData.filter(cd => shouldRemove(cd.plan)) this.synchronized { @@ -187,7 +191,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { // will keep it as it is. It means the physical plan has been re-compiled already in the // other thread. val cacheAlreadyLoaded = cd.cachedRepresentation.cacheBuilder.isCachedColumnBuffersLoaded - cd.plan.find(_.sameResult(plan)).isDefined && !cacheAlreadyLoaded + cd.plan.find(_.sameResult(analyzedPlan)).isDefined && !cacheAlreadyLoaded }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index b5307b6a82313..9cd840697be70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -122,18 +122,19 @@ case class CreateViewCommand( CommandUtils.uncacheTableOrView(sparkSession, name.quotedString) } val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) - val catalogTable = prepareTemporaryView( - name, - sparkSession, - analyzedPlan, - aliasedPlan.schema, - originalText, - child) // If there is no sql text (e.g. from Dataset API), we will always store the analyzed plan val tableDefinition = if (!conf.storeAnalyzedPlanForView && originalText.nonEmpty) { - TemporaryViewRelation(catalogTable) + TemporaryViewRelation( + prepareTemporaryView( + name, + sparkSession, + analyzedPlan, + aliasedPlan.schema, + originalText, + child)) } else { - View(catalogTable, isTemporary, catalogTable.schema.toAttributes, child) + assert(isTemporary) + View(None, isTemporary, aliasedPlan.output, aliasedPlan) } catalog.createTempView(name.table, tableDefinition, overrideIfExists = replace) } else if (viewType == GlobalTempView) { @@ -146,17 +147,18 @@ case class CreateViewCommand( CommandUtils.uncacheTableOrView(sparkSession, viewIdent.quotedString) } val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) - val catalogTable = prepareTemporaryView( - viewIdent, - sparkSession, - analyzedPlan, - aliasedPlan.schema, - originalText, - child) val tableDefinition = if (!conf.storeAnalyzedPlanForView && originalText.nonEmpty) { - TemporaryViewRelation(catalogTable) + TemporaryViewRelation( + prepareTemporaryView( + viewIdent, + sparkSession, + analyzedPlan, + aliasedPlan.schema, + originalText, + child)) } else { - View(catalogTable, isTemporary, catalogTable.schema.toAttributes, child) + assert(isTemporary) + View(None, isTemporary, aliasedPlan.output, aliasedPlan) } catalog.createGlobalTempView(name.table, tableDefinition, overrideIfExists = replace) } else if (catalog.tableExists(name)) { @@ -495,8 +497,8 @@ object ViewHelper { path: Seq[TableIdentifier], viewIdent: TableIdentifier): Unit = { plan match { - case v: View => - val ident = v.desc.identifier + case v @ View(Some(desc), _, _, _) => + val ident = desc.identifier val newPath = path :+ ident // If the table identifier equals to the `viewIdent`, current view node is the same with // the altered view. We detect a view reference cycle, should throw an AnalysisException. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index e5f02d8abc263..c72e9fc023d92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -424,7 +424,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { // same way as how a permanent view is handled. This also avoids a potential issue where a // dependent view becomes invalid because of the above while its data is still cached. val viewText = viewDef match { - case v: View => v.desc.viewText + case View(Some(desc), _, _, _) => desc.viewText case _ => None } val plan = sparkSession.sessionState.executePlan(viewDef) From 9085d1787e9b816d99c6425b0dcf9369580dfa9e Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 26 Jan 2021 09:36:18 -0800 Subject: [PATCH 05/30] test update --- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../spark/sql/execution/RemoveRedundantProjectsSuite.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 11289c15dc8b4..05690c4673efe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1548,7 +1548,7 @@ class DataFrameSuite extends QueryTest val e2 = intercept[AnalysisException] { insertion.write.insertInto("indirect_ds") } - assert(e2.getMessage.contains("Inserting into a view is not allowed. View: `indirect_ds`")) + assert(e2.getMessage.contains("Inserting into an RDD-based table is not allowed.")) // error case: insert into an OneRowRelation Dataset.ofRows(spark, OneRowRelation()).createOrReplaceTempView("one_row") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala index 040c5189abcb6..38f7a985489d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -58,6 +58,7 @@ abstract class RemoveRedundantProjectsSuiteBase "cast(id * 3 as int) as b", "cast(id as string) as c", "array(id, id + 1, id + 3) as d") .write.partitionBy("key").parquet(path) spark.read.parquet(path).createOrReplaceTempView("testView") + spark.read.parquet(path).createOrReplaceTempView("testView2") } override def afterAll(): Unit = { @@ -103,7 +104,7 @@ abstract class RemoveRedundantProjectsSuiteBase test("join without ordering requirement") { val query = "select t1.key, t2.key, t1.a, t2.b from (select key, a, b, c from testView)" + - " as t1 join (select key, a, b, c from testView) as t2 on t1.c > t2.c and t1.key > 10" + " as t1 join (select key, a, b, c from testView2) as t2 on t1.c > t2.c and t1.key > 10" assertProjectExec(query, 1, 3) } @@ -205,7 +206,7 @@ abstract class RemoveRedundantProjectsSuiteBase |SELECT t1.key, t2.key, sum(t1.a) AS s1, sum(t2.b) AS s2 FROM |(SELECT a, key FROM testView) t1 |JOIN - |(SELECT b, key FROM testView) t2 + |(SELECT b, key FROM testView2) t2 |ON t1.key = t2.key |GROUP BY t1.key, t2.key GROUPING SETS(t1.key, t2.key) |ORDER BY t1.key, t2.key, s1, s2 From bfabe9f60e472d058aca3fc9837431c960f9380c Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 28 Jan 2021 13:19:59 -0800 Subject: [PATCH 06/30] test fix --- .../org/apache/spark/sql/catalyst/analysis/view.scala | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala index f31f34a543c1e..ed2e4e7b7dcff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Alias -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} +import org.apache.spark.sql.catalyst.plans.logical.{AnalysisHelper, LogicalPlan, Project, View} import org.apache.spark.sql.catalyst.rules.Rule /** @@ -53,7 +53,14 @@ import org.apache.spark.sql.catalyst.rules.Rule * completely resolved during the batch of Resolution. */ object EliminateView extends Rule[LogicalPlan] with CastSupport { - override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + // Temporary fix until https://github.com/apache/spark/pull/31368 is merged. + override def apply(plan: LogicalPlan): LogicalPlan = { + AnalysisHelper.allowInvokingTransformsInAnalyzer { + applyInternal(plan) + } + } + + private def applyInternal(plan: LogicalPlan): LogicalPlan = plan transformUp { // The child has the different output attributes with the View operator. Adds a Project over // the child of the view. case v @ View(Some(desc), _, output, child) if child.resolved && !v.sameOutput(child) => From 1f2e4c7ae10eeebca24f35ad3e763388f2f29752 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 28 Jan 2021 18:21:15 -0800 Subject: [PATCH 07/30] Fix explain-aqe.sql output --- .../src/test/resources/sql-tests/results/explain-aqe.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index eb4384f7b526e..0b6dea1d1cf58 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -934,7 +934,7 @@ Schema: root Arguments: spark_catalog.default.explain_temp1 (5) Project -Arguments: ['key, 'val] +Arguments: [key#x, val#x] -- !query From 0b75fdc5898eb4bac6141eeaae9096e9853d51fd Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 28 Jan 2021 21:11:05 -0800 Subject: [PATCH 08/30] fix test --- .../sql/catalyst/analysis/UnsupportedOperationChecker.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index ab7d90098bfd3..bc7878f89aef8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -394,6 +394,7 @@ object UnsupportedOperationChecker extends Logging { _: DeserializeToObject | _: SerializeFromObject | _: SubqueryAlias | _: TypedFilter) => case node if node.nodeName == "StreamingRelationV2" => + case v: View if v.desc == None && v.isTempView => // Skip dataframe temp views. case node => throwError(s"Continuous processing does not support ${node.nodeName} operations.") } From 30e750c9686d7fce37103f65446d585b16dd87a4 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 30 Jan 2021 18:03:45 -0800 Subject: [PATCH 09/30] fix explain.sql --- sql/core/src/test/resources/sql-tests/results/explain.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index f92c0c6df1f09..e6fa3ea0b54f3 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -879,7 +879,7 @@ Schema: root Arguments: spark_catalog.default.explain_temp1 (5) Project -Arguments: ['key, 'val] +Arguments: [key#x, val#x] -- !query From 0fbf1cf58032e923e468c5c51c165535d8d7e690 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 30 Jan 2021 18:28:53 -0800 Subject: [PATCH 10/30] Fix tests --- .../org/apache/spark/sql/execution/CacheManager.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index bb20c03c9893a..2e9650ecbf5bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -159,9 +159,13 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { plan: LogicalPlan, cascade: Boolean, blocking: Boolean = false): Unit = { - val qe = spark.sessionState.executePlan(plan) - qe.assertAnalyzed() - val analyzedPlan = qe.analyzed + val analyzedPlan = if (plan.resolved) { + plan + } else { + val qe = spark.sessionState.executePlan(plan) + qe.assertAnalyzed() + qe.analyzed + } val shouldRemove: LogicalPlan => Boolean = if (cascade) { From a8581c6cbe279179df486e2fd2c8b1e1b9fd8a96 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 30 Jan 2021 20:32:02 -0800 Subject: [PATCH 11/30] revert RemoveRedundantProjectsSuite.scala test --- .../spark/sql/execution/RemoveRedundantProjectsSuite.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala index 38f7a985489d6..040c5189abcb6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -58,7 +58,6 @@ abstract class RemoveRedundantProjectsSuiteBase "cast(id * 3 as int) as b", "cast(id as string) as c", "array(id, id + 1, id + 3) as d") .write.partitionBy("key").parquet(path) spark.read.parquet(path).createOrReplaceTempView("testView") - spark.read.parquet(path).createOrReplaceTempView("testView2") } override def afterAll(): Unit = { @@ -104,7 +103,7 @@ abstract class RemoveRedundantProjectsSuiteBase test("join without ordering requirement") { val query = "select t1.key, t2.key, t1.a, t2.b from (select key, a, b, c from testView)" + - " as t1 join (select key, a, b, c from testView2) as t2 on t1.c > t2.c and t1.key > 10" + " as t1 join (select key, a, b, c from testView) as t2 on t1.c > t2.c and t1.key > 10" assertProjectExec(query, 1, 3) } @@ -206,7 +205,7 @@ abstract class RemoveRedundantProjectsSuiteBase |SELECT t1.key, t2.key, sum(t1.a) AS s1, sum(t2.b) AS s2 FROM |(SELECT a, key FROM testView) t1 |JOIN - |(SELECT b, key FROM testView2) t2 + |(SELECT b, key FROM testView) t2 |ON t1.key = t2.key |GROUP BY t1.key, t2.key GROUPING SETS(t1.key, t2.key) |ORDER BY t1.key, t2.key, s1, s2 From 4e77301c382807f9f2c775f9e59e0798b0dd0548 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 30 Jan 2021 22:39:52 -0800 Subject: [PATCH 12/30] rework on the udf --- .../spark/sql/catalyst/expressions/Expression.scala | 4 +++- .../apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 4 +++- .../scala/org/apache/spark/sql/UDFRegistration.scala | 2 +- .../org/apache/spark/sql/execution/aggregate/udaf.scala | 9 +++++++-- .../org/apache/spark/sql/execution/command/views.scala | 8 ++------ 5 files changed, 16 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index fa9b7d2c4252d..83f77a6abd490 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -1088,4 +1088,6 @@ trait ComplexTypeMergingExpression extends Expression { * Common base trait for user-defined functions, including UDF/UDAF/UDTF of different languages * and Hive function wrappers. */ -trait UserDefinedExpression +trait UserDefinedExpression { + def name: String +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 011a3503dd0c2..21888c5b819ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -55,9 +55,11 @@ case class ScalaUDF( udfDeterministic: Boolean = true) extends Expression with NonSQLExpression with UserDefinedExpression { + override def name: String = udfName.getOrElse("UDF") + override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) - override def toString: String = s"${udfName.getOrElse("UDF")}(${children.mkString(", ")})" + override def toString: String = s"$name(${children.mkString(", ")})" override lazy val canonicalized: Expression = { // SPARK-32307: `ExpressionEncoder` can't be canonicalized, and technically we don't diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 73131438279fa..e559299a4f604 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -82,7 +82,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends @deprecated("Aggregator[IN, BUF, OUT] should now be registered as a UDF" + " via the functions.udaf(agg) method.", "3.0.0") def register(name: String, udaf: UserDefinedAggregateFunction): UserDefinedAggregateFunction = { - def builder(children: Seq[Expression]) = ScalaUDAF(children, udaf, name = Some(name)) + def builder(children: Seq[Expression]) = ScalaUDAF(children, udaf, udafName = Some(name)) functionRegistry.createOrReplaceTempFunction(name, builder) udaf } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index 043a8d0b648a7..6f2f7570946ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -326,13 +326,15 @@ case class ScalaUDAF( udaf: UserDefinedAggregateFunction, mutableAggBufferOffset: Int = 0, inputAggBufferOffset: Int = 0, - name: Option[String] = None) + udafName: Option[String] = None) extends ImperativeAggregate with NonSQLExpression with Logging with ImplicitCastInputTypes with UserDefinedExpression { + override def name: String = udafName.getOrElse("UDAF") + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = copy(mutableAggBufferOffset = newMutableAggBufferOffset) @@ -462,7 +464,8 @@ case class ScalaAggregator[IN, BUF, OUT]( nullable: Boolean = true, isDeterministic: Boolean = true, mutableAggBufferOffset: Int = 0, - inputAggBufferOffset: Int = 0) + inputAggBufferOffset: Int = 0, + aggregatorName: Option[String] = None) extends TypedImperativeAggregate[BUF] with NonSQLExpression with UserDefinedExpression @@ -476,6 +479,8 @@ case class ScalaAggregator[IN, BUF, OUT]( private[this] lazy val outputEncoder = agg.outputEncoder.asInstanceOf[ExpressionEncoder[OUT]] private[this] lazy val outputSerializer = outputEncoder.createSerializer() + override def name: String = aggregatorName.getOrElse(nodeName) + def dataType: DataType = outputEncoder.objSerializer.dataType def inputTypes: Seq[DataType] = inputEncoder.schema.map(_.dataType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 38775a8995c08..b7628b0b025f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -26,12 +26,11 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, ViewType} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, SessionCatalog, TemporaryViewRelation} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, PythonUDF, ScalaUDF, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, SubqueryExpression, UserDefinedExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper -import org.apache.spark.sql.execution.aggregate.ScalaUDAF import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType, StructType} import org.apache.spark.sql.util.SchemaUtils @@ -576,10 +575,7 @@ object ViewHelper { child.collect { case plan if plan.resolved => plan.expressions.flatMap(_.collect { case e: SubqueryExpression => collectTempFunctions(e.plan) - case e: PythonUDF if isTemporaryFunction(e.name) => Seq(e.name) - case e: ScalaUDAF if e.name.exists(name => isTemporaryFunction(name)) => Seq(e.name.get) - case e: ScalaUDF if e.udfName.exists(name => isTemporaryFunction(name)) => - Seq(e.udfName.get) + case e: UserDefinedExpression if isTemporaryFunction(e.name) => Seq(e.name) }).flatten }.flatten.distinct } From 60cf5211fe0aee7d31c47a6c8fd269b3ca9fdd1d Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sun, 31 Jan 2021 18:21:45 -0800 Subject: [PATCH 13/30] clean up --- .../sql/catalyst/analysis/Analyzer.scala | 12 +---------- .../UnsupportedOperationChecker.scala | 1 - .../plans/logical/basicLogicalOperators.scala | 21 ++++++++++--------- .../spark/sql/execution/command/views.scala | 11 +++++----- 4 files changed, 17 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 14f083ffec002..8fc20b7213602 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -889,7 +889,6 @@ class Analyzer(override val catalogManager: CatalogManager) case UnresolvedRelation(ident, _, false) => lookupTempView(ident) .map(EliminateSubqueryAliases(_)) - .map(EliminateDataFrameTempViews(_)) .map { case r: DataSourceV2Relation => write.withNewTable(r) case _ => @@ -1092,10 +1091,6 @@ class Analyzer(override val catalogManager: CatalogManager) } } view.copy(child = newChild) - // If view.desc is None, view must be storing a dataframe temp view. - case view @ View(None, isTempView, child) => - assert(isTempView && child.resolved) - view case p @ SubqueryAlias(_, view: View) => p.copy(child = resolveViews(view)) case _ => plan @@ -1110,11 +1105,6 @@ class Analyzer(override val catalogManager: CatalogManager) } EliminateSubqueryAliases(relation) match { - case v: View if v.isTempView && v.child.isInstanceOf[LeafNode] => - // Inserting into a file-based temporary view is allowed. - // (e.g., spark.read.parquet("path").createOrReplaceTempView("t"). - // The check on the rdd-based relation is done in PreWriteCheck. - i.copy(table = v.child) case v: View if v.desc.isDefined => throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.get.identifier, table) case other => i.copy(table = other) @@ -3684,7 +3674,7 @@ object EliminateSubqueryAliases extends Rule[LogicalPlan] { } /** - * Removes [[View]] operators from the plan if they are temp views created by DataFrame API. + * Removes [[View]] operators from the plan if they are temp views created from dataframes. */ object EliminateDataFrameTempViews extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index bc7878f89aef8..ab7d90098bfd3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -394,7 +394,6 @@ object UnsupportedOperationChecker extends Logging { _: DeserializeToObject | _: SerializeFromObject | _: SubqueryAlias | _: TypedFilter) => case node if node.nodeName == "StreamingRelationV2" => - case v: View if v.desc == None && v.isTempView => // Skip dataframe temp views. case node => throwError(s"Continuous processing does not support ${node.nodeName} operations.") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 2155a228cbfbd..7a1548437d1cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -443,27 +443,28 @@ case class InsertIntoDir( } /** - * A container for holding the view description(CatalogTable), and the output of the view. The - * child should be a logical plan parsed from the `CatalogTable.viewText`, should throw an error - * if the `viewText` is not defined. + * A container for holding the view description(CatalogTable) and info whether the view is temporary + * or not. If the view description is available, the child should be a logical plan parsed from the + * `CatalogTable.viewText`. Otherwise, the view is a temporary one created from a dataframe; in this + * case, the child must be already resolved. * This operator will be removed at the end of analysis stage. * - * @param desc A view description(CatalogTable) that provides necessary information to resolve the - * view. - * we are able to decouple the output from the underlying structure. - * @param child The logical plan of a view operator, it should be a logical plan parsed from the - * `CatalogTable.viewText`, should throw an error if the `viewText` is not defined. + * @param desc An optional view description(CatalogTable) that provides necessary information to + * resolve the view. + * @param isTempView A flag to indicate whether the view is temporary or not. + * @param child The logical plan of a view operator. If the view description is available, it should + * be a logical plan parsed from the `CatalogTable.viewText`. */ case class View( desc: Option[CatalogTable], isTempView: Boolean, child: LogicalPlan) extends UnaryNode { - require(desc.isDefined || isTempView) + require(desc.isDefined || (isTempView && child.resolved)) override def output: Seq[Attribute] = child.output override def simpleString(maxFields: Int): String = { - val viewIdent = desc.map{ d => s"${d.identifier}, "}.getOrElse("") + val viewIdent = desc.map(d => s"${d.identifier}, ").getOrElse("") s"View ($viewIdent${output.mkString("[", ",", "]")})" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index b7628b0b025f1..b54ce4783d262 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -562,20 +562,19 @@ object ViewHelper { child.collect { case s @ SubqueryAlias(_, view: View) if view.isTempView => Seq(s.identifier.qualifier :+ s.identifier.name) - case plan if plan.resolved => plan.expressions.flatMap(_.collect { + case plan => plan.expressions.flatMap(_.collect { case e: SubqueryExpression => collectTempViews(e.plan) }).flatten }.flatten.distinct } def collectTempFunctions(child: LogicalPlan): Seq[String] = { - def isTemporaryFunction(name: String): Boolean = { - catalog.isTemporaryFunction(FunctionIdentifier(name)) - } child.collect { - case plan if plan.resolved => plan.expressions.flatMap(_.collect { + case plan => plan.expressions.flatMap(_.collect { case e: SubqueryExpression => collectTempFunctions(e.plan) - case e: UserDefinedExpression if isTemporaryFunction(e.name) => Seq(e.name) + case e: UserDefinedExpression + if catalog.isTemporaryFunction(FunctionIdentifier(e.name)) => + Seq(e.name) }).flatten }.flatten.distinct } From a0ba50895c5aafeb449505d952dd314a5ce7b594 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sun, 31 Jan 2021 21:32:08 -0800 Subject: [PATCH 14/30] clean up --- .../spark/sql/catalyst/analysis/Analyzer.scala | 12 +++++++++--- .../analysis/UnsupportedOperationChecker.scala | 1 + .../plans/logical/basicLogicalOperators.scala | 2 ++ 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8fc20b7213602..beba738a0b184 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -889,6 +889,7 @@ class Analyzer(override val catalogManager: CatalogManager) case UnresolvedRelation(ident, _, false) => lookupTempView(ident) .map(EliminateSubqueryAliases(_)) + .map(EliminateDataFrameTempViews(_)) .map { case r: DataSourceV2Relation => write.withNewTable(r) case _ => @@ -1104,7 +1105,10 @@ class Analyzer(override val catalogManager: CatalogManager) case other => other } - EliminateSubqueryAliases(relation) match { + // Inserting into a file-based temporary view is allowed. + // (e.g., spark.read.parquet("path").createOrReplaceTempView("t"). + // Thus, apply EliminateDataFrameTempViews rule to remove any dataframe temp views. + EliminateDataFrameTempViews(EliminateSubqueryAliases(relation)) match { case v: View if v.desc.isDefined => throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.get.identifier, table) case other => i.copy(table = other) @@ -3677,8 +3681,10 @@ object EliminateSubqueryAliases extends Rule[LogicalPlan] { * Removes [[View]] operators from the plan if they are temp views created from dataframes. */ object EliminateDataFrameTempViews extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case v: View if v.desc.isEmpty => v.child + def apply(plan: LogicalPlan): LogicalPlan = AnalysisHelper.allowInvokingTransformsInAnalyzer { + plan transformUp { + case v: View if v.isDataFrameTempView => v.child + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index ab7d90098bfd3..42ccc45cec62f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -393,6 +393,7 @@ object UnsupportedOperationChecker extends Logging { case (_: Project | _: Filter | _: MapElements | _: MapPartitions | _: DeserializeToObject | _: SerializeFromObject | _: SubqueryAlias | _: TypedFilter) => + case v: View if v.isDataFrameTempView => case node if node.nodeName == "StreamingRelationV2" => case node => throwError(s"Continuous processing does not support ${node.nodeName} operations.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 7a1548437d1cc..9457bf8ea4257 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -473,6 +473,8 @@ case class View( case _ => child.canonicalized } + def isDataFrameTempView: Boolean = desc.isEmpty && isTempView + // When resolving a SQL view, we use an extra Project to add cast and alias to make sure the view // output schema doesn't change even if the table referenced by the view is changed after view // creation. We should remove this extra Project during canonicalize if it does nothing. From 388ec16ba80b136a0bebbbe5adb6af7329c99df9 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 12 Feb 2021 09:31:02 -0800 Subject: [PATCH 15/30] Fix build issue --- .../scala/org/apache/spark/sql/execution/aggregate/udaf.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index b3574bcb89d8a..e6851a9af739f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -333,8 +333,6 @@ case class ScalaUDAF( with ImplicitCastInputTypes with UserDefinedExpression { - override def name: String = udafName.getOrElse("UDAF") - override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = copy(mutableAggBufferOffset = newMutableAggBufferOffset) @@ -481,8 +479,6 @@ case class ScalaAggregator[IN, BUF, OUT]( private[this] lazy val outputEncoder = agg.outputEncoder.asInstanceOf[ExpressionEncoder[OUT]] private[this] lazy val outputSerializer = outputEncoder.createSerializer() - override def name: String = aggregatorName.getOrElse(nodeName) - def dataType: DataType = outputEncoder.objSerializer.dataType def inputTypes: Seq[DataType] = inputEncoder.schema.map(_.dataType) From 7e92eeb88aebba1fae2e6e94bc4e678998118fea Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 12 Feb 2021 11:59:04 -0800 Subject: [PATCH 16/30] fix --- .../spark/sql/catalyst/analysis/Analyzer.scala | 15 ++++++++++----- .../apache/spark/sql/execution/CacheManager.scala | 14 +++----------- 2 files changed, 13 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4ff1a3eb5bd29..76bb8115987ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -116,7 +116,8 @@ case class AnalysisContext( maxNestedViewDepth: Int = -1, relationCache: mutable.Map[Seq[String], LogicalPlan] = mutable.Map.empty, referredTempViewNames: Seq[Seq[String]] = Seq.empty, - referredTempFunctionNames: Seq[String] = Seq.empty) + referredTempFunctionNames: Seq[String] = Seq.empty, + isTempView: Boolean = false) object AnalysisContext { private val value = new ThreadLocal[AnalysisContext]() { @@ -128,7 +129,7 @@ object AnalysisContext { private def set(context: AnalysisContext): Unit = value.set(context) - def withAnalysisContext[A](viewDesc: CatalogTable)(f: => A): A = { + def withAnalysisContext[A](viewDesc: CatalogTable, isTempView: Boolean)(f: => A): A = { val originContext = value.get() val maxNestedViewDepth = if (originContext.maxNestedViewDepth == -1) { // Here we start to resolve views, get `maxNestedViewDepth` from configs. @@ -142,7 +143,8 @@ object AnalysisContext { maxNestedViewDepth, originContext.relationCache, viewDesc.viewReferredTempViewNames, - viewDesc.viewReferredTempFunctionNames) + viewDesc.viewReferredTempFunctionNames, + isTempView) set(context) try f finally { set(originContext) } } @@ -861,6 +863,7 @@ class Analyzer(override val catalogManager: CatalogManager) } private def isResolvingView: Boolean = AnalysisContext.get.catalogAndNamespace.nonEmpty + private def isResolvingTempView: Boolean = AnalysisContext.get.isTempView private def referredTempViewNames: Seq[Seq[String]] = AnalysisContext.get.referredTempViewNames /** @@ -926,7 +929,9 @@ class Analyzer(override val catalogManager: CatalogManager) def lookupTempView( identifier: Seq[String], isStreaming: Boolean = false): Option[LogicalPlan] = { // Permanent View can't refer to temp views, no need to lookup at all. - if (isResolvingView && !referredTempViewNames.contains(identifier)) return None + if (isResolvingView && !isResolvingTempView && !referredTempViewNames.contains(identifier)) { + return None + } val tmpView = identifier match { case Seq(part1) => v1SessionCatalog.lookupTempView(part1) @@ -1107,7 +1112,7 @@ class Analyzer(override val catalogManager: CatalogManager) // operator. case view @ View(Some(desc), isTempView, child) if !child.resolved => // Resolve all the UnresolvedRelations and Views in the child. - val newChild = AnalysisContext.withAnalysisContext(desc) { + val newChild = AnalysisContext.withAnalysisContext(desc, isTempView) { val nestedViewDepth = AnalysisContext.get.nestedViewDepth val maxNestedViewDepth = AnalysisContext.get.maxNestedViewDepth if (nestedViewDepth > maxNestedViewDepth) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 2e9650ecbf5bf..0c6f22dea7b45 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -159,19 +159,11 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { plan: LogicalPlan, cascade: Boolean, blocking: Boolean = false): Unit = { - val analyzedPlan = if (plan.resolved) { - plan - } else { - val qe = spark.sessionState.executePlan(plan) - qe.assertAnalyzed() - qe.analyzed - } - val shouldRemove: LogicalPlan => Boolean = if (cascade) { - _.find(_.sameResult(analyzedPlan)).isDefined + _.find(_.sameResult(plan)).isDefined } else { - _.sameResult(analyzedPlan) + _.sameResult(plan) } val plansToUncache = cachedData.filter(cd => shouldRemove(cd.plan)) this.synchronized { @@ -195,7 +187,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { // will keep it as it is. It means the physical plan has been re-compiled already in the // other thread. val cacheAlreadyLoaded = cd.cachedRepresentation.cacheBuilder.isCachedColumnBuffersLoaded - cd.plan.find(_.sameResult(analyzedPlan)).isDefined && !cacheAlreadyLoaded + cd.plan.find(_.sameResult(plan)).isDefined && !cacheAlreadyLoaded }) } } From fbdcf3b7535392ef0cd02d7ab32cbb3e5f8116b4 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 12 Feb 2021 16:20:19 -0800 Subject: [PATCH 17/30] fix test --- .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 76bb8115987ce..da1ace1d76cf6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -933,7 +933,7 @@ class Analyzer(override val catalogManager: CatalogManager) return None } - val tmpView = identifier match { + val tmpView = expandRelationName(identifier) match { case Seq(part1) => v1SessionCatalog.lookupTempView(part1) case Seq(part1, part2) => v1SessionCatalog.lookupGlobalTempView(part1, part2) case _ => None From 2a3e5b74cfa5277fd3b72db38b205901474294e6 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 12 Feb 2021 19:15:35 -0800 Subject: [PATCH 18/30] Fix test --- .../spark/sql/catalyst/analysis/Analyzer.scala | 10 +++++++++- .../apache/spark/sql/execution/command/views.scala | 13 +++++-------- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index da1ace1d76cf6..636e2dc041297 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -949,7 +949,15 @@ class Analyzer(override val catalogManager: CatalogManager) // If we are resolving relations insides views, we need to expand single-part relation names with // the current catalog and namespace of when the view was created. private def expandRelationName(nameParts: Seq[String]): Seq[String] = { - if (!isResolvingView || referredTempViewNames.contains(nameParts)) return nameParts + def isReferredTempViewName(nameParts: Seq[String]): Boolean = { + referredTempViewNames.exists { n => + (n.length == nameParts.length) && n.zip(nameParts).forall { + case (a, b) => conf.resolver(a, b) + } + } + } + + if (!isResolvingView || isReferredTempViewName(nameParts)) return nameParts if (nameParts.length == 1) { AnalysisContext.get.catalogAndNamespace :+ nameParts.head diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index b54ce4783d262..9d24f9633bc31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -129,8 +129,7 @@ case class CreateViewCommand( sparkSession, analyzedPlan, aliasedPlan.schema, - originalText, - child)) + originalText)) } else { assert(isTemporary) View(None, isTemporary, aliasedPlan) @@ -153,8 +152,7 @@ case class CreateViewCommand( sparkSession, analyzedPlan, aliasedPlan.schema, - originalText, - child)) + originalText)) } else { assert(isTemporary) View(None, isTemporary, aliasedPlan) @@ -282,7 +280,7 @@ case class AlterViewAsCommand( checkCyclicViewReference(analyzedPlan, Seq(name), name) TemporaryViewRelation( prepareTemporaryView( - name, session, analyzedPlan, analyzedPlan.schema, Some(originalText), query)) + name, session, analyzedPlan, analyzedPlan.schema, Some(originalText))) } session.sessionState.catalog.alterTempViewDefinition(name, tableDefinition) } @@ -593,11 +591,10 @@ object ViewHelper { session: SparkSession, analyzedPlan: LogicalPlan, viewSchema: StructType, - originalText: Option[String], - child: LogicalPlan): CatalogTable = { + originalText: Option[String]): CatalogTable = { val catalog = session.sessionState.catalog - val (tempViews, tempFunctions) = collectTemporaryObjects(catalog, child) + val (tempViews, tempFunctions) = collectTemporaryObjects(catalog, analyzedPlan) // TBLPROPERTIES is not allowed for temporary view, so we don't use it for // generating temporary view properties val newProperties = generateViewProperties( From e92cbe1427dacb8f6b1f7e967c4d1c6a75c32ebf Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 13 Feb 2021 10:02:04 -0800 Subject: [PATCH 19/30] refinement --- .../spark/sql/catalyst/analysis/Analyzer.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 636e2dc041297..0b21b7e9a03a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -864,7 +864,13 @@ class Analyzer(override val catalogManager: CatalogManager) private def isResolvingView: Boolean = AnalysisContext.get.catalogAndNamespace.nonEmpty private def isResolvingTempView: Boolean = AnalysisContext.get.isTempView - private def referredTempViewNames: Seq[Seq[String]] = AnalysisContext.get.referredTempViewNames + private def isReferredTempViewName(nameParts: Seq[String]): Boolean = { + AnalysisContext.get.referredTempViewNames.exists { n => + (n.length == nameParts.length) && n.zip(nameParts).forall { + case (a, b) => conf.resolver(a, b) + } + } + } /** * Resolve relations to temp views. This is not an actual rule, and is called by @@ -929,7 +935,7 @@ class Analyzer(override val catalogManager: CatalogManager) def lookupTempView( identifier: Seq[String], isStreaming: Boolean = false): Option[LogicalPlan] = { // Permanent View can't refer to temp views, no need to lookup at all. - if (isResolvingView && !isResolvingTempView && !referredTempViewNames.contains(identifier)) { + if (isResolvingView && !isResolvingTempView && !isReferredTempViewName(identifier)) { return None } @@ -949,14 +955,6 @@ class Analyzer(override val catalogManager: CatalogManager) // If we are resolving relations insides views, we need to expand single-part relation names with // the current catalog and namespace of when the view was created. private def expandRelationName(nameParts: Seq[String]): Seq[String] = { - def isReferredTempViewName(nameParts: Seq[String]): Boolean = { - referredTempViewNames.exists { n => - (n.length == nameParts.length) && n.zip(nameParts).forall { - case (a, b) => conf.resolver(a, b) - } - } - } - if (!isResolvingView || isReferredTempViewName(nameParts)) return nameParts if (nameParts.length == 1) { From 8422ae8677910322d7d86e90c253d2990a6a04cb Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 18 Feb 2021 12:51:01 -0800 Subject: [PATCH 20/30] revert AnalysisContext change --- .../spark/sql/catalyst/analysis/Analyzer.scala | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0b21b7e9a03a4..fb63a3bde5dd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -116,8 +116,7 @@ case class AnalysisContext( maxNestedViewDepth: Int = -1, relationCache: mutable.Map[Seq[String], LogicalPlan] = mutable.Map.empty, referredTempViewNames: Seq[Seq[String]] = Seq.empty, - referredTempFunctionNames: Seq[String] = Seq.empty, - isTempView: Boolean = false) + referredTempFunctionNames: Seq[String] = Seq.empty) object AnalysisContext { private val value = new ThreadLocal[AnalysisContext]() { @@ -129,7 +128,7 @@ object AnalysisContext { private def set(context: AnalysisContext): Unit = value.set(context) - def withAnalysisContext[A](viewDesc: CatalogTable, isTempView: Boolean)(f: => A): A = { + def withAnalysisContext[A](viewDesc: CatalogTable)(f: => A): A = { val originContext = value.get() val maxNestedViewDepth = if (originContext.maxNestedViewDepth == -1) { // Here we start to resolve views, get `maxNestedViewDepth` from configs. @@ -143,8 +142,7 @@ object AnalysisContext { maxNestedViewDepth, originContext.relationCache, viewDesc.viewReferredTempViewNames, - viewDesc.viewReferredTempFunctionNames, - isTempView) + viewDesc.viewReferredTempFunctionNames) set(context) try f finally { set(originContext) } } @@ -863,7 +861,6 @@ class Analyzer(override val catalogManager: CatalogManager) } private def isResolvingView: Boolean = AnalysisContext.get.catalogAndNamespace.nonEmpty - private def isResolvingTempView: Boolean = AnalysisContext.get.isTempView private def isReferredTempViewName(nameParts: Seq[String]): Boolean = { AnalysisContext.get.referredTempViewNames.exists { n => (n.length == nameParts.length) && n.zip(nameParts).forall { @@ -935,7 +932,7 @@ class Analyzer(override val catalogManager: CatalogManager) def lookupTempView( identifier: Seq[String], isStreaming: Boolean = false): Option[LogicalPlan] = { // Permanent View can't refer to temp views, no need to lookup at all. - if (isResolvingView && !isResolvingTempView && !isReferredTempViewName(identifier)) { + if (isResolvingView && !isReferredTempViewName(identifier)) { return None } @@ -1118,7 +1115,7 @@ class Analyzer(override val catalogManager: CatalogManager) // operator. case view @ View(Some(desc), isTempView, child) if !child.resolved => // Resolve all the UnresolvedRelations and Views in the child. - val newChild = AnalysisContext.withAnalysisContext(desc, isTempView) { + val newChild = AnalysisContext.withAnalysisContext(desc) { val nestedViewDepth = AnalysisContext.get.nestedViewDepth val maxNestedViewDepth = AnalysisContext.get.maxNestedViewDepth if (nestedViewDepth > maxNestedViewDepth) { From bace91a680e093ef4828495ee16a931769f2aca2 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 18 Feb 2021 12:52:08 -0800 Subject: [PATCH 21/30] clean up --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fb63a3bde5dd6..e4a47f2c29fcd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -932,9 +932,7 @@ class Analyzer(override val catalogManager: CatalogManager) def lookupTempView( identifier: Seq[String], isStreaming: Boolean = false): Option[LogicalPlan] = { // Permanent View can't refer to temp views, no need to lookup at all. - if (isResolvingView && !isReferredTempViewName(identifier)) { - return None - } + if (isResolvingView && !isReferredTempViewName(identifier)) return None val tmpView = expandRelationName(identifier) match { case Seq(part1) => v1SessionCatalog.lookupTempView(part1) From 737307cc3be7fea2f6ba184657eede034c3f3242 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 19 Feb 2021 13:58:47 -0800 Subject: [PATCH 22/30] Address PR comments --- .../spark/sql/catalyst/analysis/Analyzer.scala | 15 +++++++++------ .../sql/catalyst/catalog/SessionCatalog.scala | 11 +++-------- .../spark/sql/catalyst/optimizer/Optimizer.scala | 1 - .../spark/sql/execution/command/views.scala | 16 ++++------------ 4 files changed, 16 insertions(+), 27 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 72f597a5f42c2..49716163e4c78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -865,11 +865,15 @@ class Analyzer(override val catalogManager: CatalogManager) private def isReferredTempViewName(nameParts: Seq[String]): Boolean = { AnalysisContext.get.referredTempViewNames.exists { n => (n.length == nameParts.length) && n.zip(nameParts).forall { - case (a, b) => conf.resolver(a, b) + case (a, b) => resolver(a, b) } } } + private def getTempViewRawPlan(plan: LogicalPlan): LogicalPlan = { + EliminateDataFrameTempViews(EliminateSubqueryAliases(plan)) + } + /** * Resolve relations to temp views. This is not an actual rule, and is called by * [[ResolveTables]] and [[ResolveRelations]]. @@ -895,8 +899,7 @@ class Analyzer(override val catalogManager: CatalogManager) write.table match { case UnresolvedRelation(ident, _, false) => lookupTempView(ident) - .map(EliminateSubqueryAliases(_)) - .map(EliminateDataFrameTempViews(_)) + .map(getTempViewRawPlan) .map { case r: DataSourceV2Relation => write.withNewTable(r) case _ => @@ -935,7 +938,7 @@ class Analyzer(override val catalogManager: CatalogManager) // Permanent View can't refer to temp views, no need to lookup at all. if (isResolvingView && !isReferredTempViewName(identifier)) return None - val tmpView = expandRelationName(identifier) match { + val tmpView = identifier match { case Seq(part1) => v1SessionCatalog.lookupTempView(part1) case Seq(part1, part2) => v1SessionCatalog.lookupGlobalTempView(part1, part2) case _ => None @@ -1141,8 +1144,8 @@ class Analyzer(override val catalogManager: CatalogManager) // Inserting into a file-based temporary view is allowed. // (e.g., spark.read.parquet("path").createOrReplaceTempView("t"). - // Thus, apply EliminateDataFrameTempViews rule to remove any dataframe temp views. - EliminateDataFrameTempViews(EliminateSubqueryAliases(relation)) match { + // Thus, we need to look at the raw plan of a temporary view. + getTempViewRawPlan(relation) match { case v: View if v.desc.isDefined => throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.get.identifier, table) case other => i.copy(table = other) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 3591410650104..67407f47c1b93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -840,14 +840,9 @@ class SessionCatalog( plan match { case viewInfo: TemporaryViewRelation => fromCatalogTable(viewInfo.tableMeta, isTempView = true) - case v => v - } - } - - def getTempViewSchema(plan: LogicalPlan): StructType = { - plan match { - case viewInfo: TemporaryViewRelation => viewInfo.tableMeta.schema - case v => v.schema + case v => + // This must be a temporary view created from dataframe. + View(None, isTempView = true, v) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 30749b07ad959..1941c9efa9975 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -143,7 +143,6 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateResolvedHint, EliminateSubqueryAliases, EliminateView, - EliminateDataFrameTempViews, ReplaceExpressions, RewriteNonCorrelatedExists, ComputeCurrentTime, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 4592f6ae4a238..1ca5d0db69ce2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -111,7 +111,7 @@ case class CreateViewCommand( // When creating a permanent view, not allowed to reference temporary objects. // This should be called after `qe.assertAnalyzed()` (i.e., `child` can be resolved) - verifyTemporaryObjectsNotExists(catalog, isTemporary, name, child) + verifyTemporaryObjectsNotExists(catalog, isTemporary, name, analyzedPlan) if (viewType == LocalTempView) { val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) @@ -131,8 +131,7 @@ case class CreateViewCommand( aliasedPlan.schema, originalText)) } else { - assert(isTemporary) - View(None, isTemporary, aliasedPlan) + aliasedPlan } catalog.createTempView(name.table, tableDefinition, overrideIfExists = replace) } else if (viewType == GlobalTempView) { @@ -154,8 +153,7 @@ case class CreateViewCommand( aliasedPlan.schema, originalText)) } else { - assert(isTemporary) - View(None, isTemporary, aliasedPlan) + aliasedPlan } catalog.createGlobalTempView(name.table, tableDefinition, overrideIfExists = replace) } else if (catalog.tableExists(name)) { @@ -541,13 +539,7 @@ object ViewHelper { } /** - * Collect all temporary views and functions and return the identifiers separately - * This func traverses the unresolved plan `child`. Below are the reasons: - * 1) Analyzer replaces unresolved temporary views by a SubqueryAlias with the corresponding - * logical plan. After replacement, it is impossible to detect whether the SubqueryAlias is - * added/generated from a temporary view. - * 2) The temp functions are represented by multiple classes. Most are inaccessible from this - * package (e.g., HiveGenericUDF). + * Collect all temporary views and functions and return the identifiers separately. */ private def collectTemporaryObjects( catalog: SessionCatalog, child: LogicalPlan): (Seq[Seq[String]], Seq[String]) = { From 4e55a1f4fac4720782670aafe3e2fa3f9252ec6b Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 19 Feb 2021 14:17:21 -0800 Subject: [PATCH 23/30] reduce churn --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 49716163e4c78..d00c07fffe5bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -898,13 +898,10 @@ class Analyzer(override val catalogManager: CatalogManager) case write: V2WriteCommand => write.table match { case UnresolvedRelation(ident, _, false) => - lookupTempView(ident) - .map(getTempViewRawPlan) - .map { - case r: DataSourceV2Relation => write.withNewTable(r) - case _ => - throw QueryCompilationErrors.writeIntoTempViewNotAllowedError(ident.quoted) - }.getOrElse(write) + lookupTempView(ident).map(getTempViewRawPlan).map { + case r: DataSourceV2Relation => write.withNewTable(r) + case _ => throw QueryCompilationErrors.writeIntoTempViewNotAllowedError(ident.quoted) + }.getOrElse(write) case _ => write } case u @ UnresolvedTable(ident, cmd, _) => From e9cd2f332a56bb47446b10fde2fa0658dd834e7e Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 19 Feb 2021 15:34:15 -0800 Subject: [PATCH 24/30] revert removing def --- .../apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 67407f47c1b93..93c794d071d76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -846,6 +846,13 @@ class SessionCatalog( } } + def getTempViewSchema(plan: LogicalPlan): StructType = { + plan match { + case viewInfo: TemporaryViewRelation => viewInfo.tableMeta.schema + case v => v.schema + } + } + private def fromCatalogTable(metadata: CatalogTable, isTempView: Boolean): View = { val viewText = metadata.viewText.getOrElse { throw new IllegalStateException("Invalid view without text.") From 9da3b9cd6ff1b6080caf0cfa7ab0021b0a079879 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 19 Feb 2021 20:05:25 -0800 Subject: [PATCH 25/30] fix tests --- .../sql/catalyst/catalog/SessionCatalog.scala | 7 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 16 ++-- .../catalyst/analysis/ResolveHintsSuite.scala | 73 ++++++++++--------- .../catalog/SessionCatalogSuite.scala | 22 +++--- 4 files changed, 62 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 93c794d071d76..191d7e6a9da3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -622,8 +622,7 @@ class SessionCatalog( } /** - * Generate a [[View]] operator from the view description if the view stores sql text, - * otherwise, it is same to `getRawTempView` + * Generate a [[View]] operator from the temporary view stored. */ def getTempView(name: String): Option[LogicalPlan] = synchronized { getRawTempView(name).map(getTempViewPlan) @@ -840,9 +839,7 @@ class SessionCatalog( plan match { case viewInfo: TemporaryViewRelation => fromCatalogTable(viewInfo.tableMeta, isTempView = true) - case v => - // This must be a temporary view created from dataframe. - View(None, isTempView = true, v) + case v => View(None, isTempView = true, v) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 0e0142eb76894..08517e324ff0f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -91,6 +91,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("analyze project") { + val testTempView = View(None, isTempView = true, testRelation) + checkAnalysis( Project(Seq(UnresolvedAttribute("a")), testRelation), Project(testRelation.output, testRelation)) @@ -98,7 +100,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkAnalysis( Project(Seq(UnresolvedAttribute("TbL.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), - Project(testRelation.output, testRelation)) + Project(testTempView.output, testTempView)) assertAnalysisError( Project(Seq(UnresolvedAttribute("tBl.a")), @@ -108,13 +110,13 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkAnalysis( Project(Seq(UnresolvedAttribute("TbL.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), - Project(testRelation.output, testRelation), + Project(testTempView.output, testTempView), caseSensitive = false) checkAnalysis( Project(Seq(UnresolvedAttribute("tBl.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), - Project(testRelation.output, testRelation), + Project(testTempView.output, testTempView), caseSensitive = false) } @@ -202,12 +204,14 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("resolve relations") { + val testTempView = View(None, isTempView = true, testRelation) + assertAnalysisError(UnresolvedRelation(TableIdentifier("tAbLe")), Seq()) - checkAnalysis(UnresolvedRelation(TableIdentifier("TaBlE")), testRelation) + checkAnalysis(UnresolvedRelation(TableIdentifier("TaBlE")), testTempView) checkAnalysis( - UnresolvedRelation(TableIdentifier("tAbLe")), testRelation, caseSensitive = false) + UnresolvedRelation(TableIdentifier("tAbLe")), testTempView, caseSensitive = false) checkAnalysis( - UnresolvedRelation(TableIdentifier("TaBlE")), testRelation, caseSensitive = false) + UnresolvedRelation(TableIdentifier("TaBlE")), testTempView, caseSensitive = false) } test("divide should be casted into fractional types") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index 513f1d001f757..c1417e5965227 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -30,40 +30,45 @@ import org.apache.spark.sql.types.IntegerType class ResolveHintsSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.analysis.TestRelations._ + val testTempView = View(None, isTempView = true, testRelation) + val testTempView2 = View(None, isTempView = true, testRelation2) + val testTempView4 = View(None, isTempView = true, testRelation4) + val testTempView5 = View(None, isTempView = true, testRelation5) + test("invalid hints should be ignored") { checkAnalysis( UnresolvedHint("some_random_hint_that_does_not_exist", Seq("TaBlE"), table("TaBlE")), - testRelation, + testTempView, caseSensitive = false) } test("case-sensitive or insensitive parameters") { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("TaBlE"), table("TaBlE")), - ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table"), table("TaBlE")), - ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) checkAnalysis( UnresolvedHint("MAPJOIN", Seq("TaBlE"), table("TaBlE")), - ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), caseSensitive = true) checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table"), table("TaBlE")), - testRelation, + testTempView, caseSensitive = true) } test("multiple broadcast hint aliases") { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table", "table2"), table("table").join(table("table2"))), - Join(ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), - ResolvedHint(testRelation2, HintInfo(strategy = Some(BROADCAST))), + Join(ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testTempView2, HintInfo(strategy = Some(BROADCAST))), Inner, None, JoinHint.NONE), caseSensitive = false) } @@ -72,32 +77,32 @@ class ResolveHintsSuite extends AnalysisTest { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table"), ResolvedHint(table("table").where('a > 1), HintInfo(strategy = Some(BROADCAST)))), - ResolvedHint(testRelation.where('a > 1), HintInfo(strategy = Some(BROADCAST))).analyze, + ResolvedHint(testTempView.where('a > 1), HintInfo(strategy = Some(BROADCAST))).analyze, caseSensitive = false) } test("should work for subqueries") { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").as("tableAlias")), - ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) checkAnalysis( UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").subquery('tableAlias)), - ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) // Negative case: if the alias doesn't match, don't match the original table name. checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table"), table("table").as("tableAlias")), - testRelation, + testTempView, caseSensitive = false) } test("do not traverse past subquery alias") { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table"), table("table").where('a > 1).subquery('tableAlias)), - testRelation.where('a > 1).analyze, + testTempView.where('a > 1).analyze, caseSensitive = false) } @@ -109,7 +114,7 @@ class ResolveHintsSuite extends AnalysisTest { |SELECT /*+ BROADCAST(ctetable) */ * FROM ctetable """.stripMargin ), - ResolvedHint(testRelation.where('a > 1).select('a), HintInfo(strategy = Some(BROADCAST))) + ResolvedHint(testTempView.where('a > 1).select('a), HintInfo(strategy = Some(BROADCAST))) .select('a).analyze, caseSensitive = false) } @@ -122,23 +127,23 @@ class ResolveHintsSuite extends AnalysisTest { |SELECT /*+ BROADCAST(table) */ * FROM ctetable """.stripMargin ), - testRelation.where('a > 1).select('a).select('a).analyze, + testTempView.where('a > 1).select('a).select('a).analyze, caseSensitive = false) } test("coalesce and repartition hint") { checkAnalysis( UnresolvedHint("COALESCE", Seq(Literal(10)), table("TaBlE")), - Repartition(numPartitions = 10, shuffle = false, child = testRelation)) + Repartition(numPartitions = 10, shuffle = false, child = testTempView)) checkAnalysis( UnresolvedHint("coalesce", Seq(Literal(20)), table("TaBlE")), - Repartition(numPartitions = 20, shuffle = false, child = testRelation)) + Repartition(numPartitions = 20, shuffle = false, child = testTempView)) checkAnalysis( UnresolvedHint("REPARTITION", Seq(Literal(100)), table("TaBlE")), - Repartition(numPartitions = 100, shuffle = true, child = testRelation)) + Repartition(numPartitions = 100, shuffle = true, child = testTempView)) checkAnalysis( UnresolvedHint("RePARTITion", Seq(Literal(200)), table("TaBlE")), - Repartition(numPartitions = 200, shuffle = true, child = testRelation)) + Repartition(numPartitions = 200, shuffle = true, child = testTempView)) val errMsg = "COALESCE Hint expects a partition number as a parameter" @@ -154,16 +159,16 @@ class ResolveHintsSuite extends AnalysisTest { checkAnalysis( UnresolvedHint("RePartition", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), - RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testRelation, 10)) + RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testTempView, 10)) checkAnalysis( UnresolvedHint("REPARTITION", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), - RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testRelation, 10)) + RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testTempView, 10)) checkAnalysis( UnresolvedHint("REPARTITION", Seq(UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( - Seq(AttributeReference("a", IntegerType)()), testRelation, None)) + Seq(AttributeReference("a", IntegerType)()), testTempView, None)) val e = intercept[IllegalArgumentException] { checkAnalysis( @@ -171,7 +176,7 @@ class ResolveHintsSuite extends AnalysisTest { Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), table("TaBlE")), RepartitionByExpression( - Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testRelation, 10) + Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testTempView, 10) ) } e.getMessage.contains("For range partitioning use REPARTITION_BY_RANGE instead") @@ -180,14 +185,14 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint( "REPARTITION_BY_RANGE", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( - Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testRelation, 10)) + Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testTempView, 10)) checkAnalysis( UnresolvedHint( "REPARTITION_BY_RANGE", Seq(UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), - testRelation, None)) + testTempView, None)) val errMsg2 = "REPARTITION Hint parameter should include columns, but" @@ -227,7 +232,7 @@ class ResolveHintsSuite extends AnalysisTest { withLogAppender(logAppender) { checkAnalysis( UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE")), - testRelation, + testTempView, caseSensitive = false) } assert(logAppender.loggingEvents.exists( @@ -238,7 +243,7 @@ class ResolveHintsSuite extends AnalysisTest { test("SPARK-30003: Do not throw stack overflow exception in non-root unknown hint resolution") { checkAnalysis( Project(testRelation.output, UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE"))), - Project(testRelation.output, testRelation), + Project(testTempView.output, testTempView), caseSensitive = false) } @@ -252,8 +257,8 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint(hintName, Seq("table", "table2"), table("TaBlE").join(table("TaBlE2"))), Join( - ResolvedHint(testRelation, HintInfo(strategy = Some(st))), - ResolvedHint(testRelation2, HintInfo(strategy = Some(st))), + ResolvedHint(testTempView, HintInfo(strategy = Some(st))), + ResolvedHint(testTempView2, HintInfo(strategy = Some(st))), Inner, None, JoinHint.NONE), @@ -263,8 +268,8 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint(hintName, Seq("TaBlE", "table2"), table("TaBlE").join(table("TaBlE2"))), Join( - ResolvedHint(testRelation, HintInfo(strategy = Some(st))), - testRelation2, + ResolvedHint(testTempView, HintInfo(strategy = Some(st))), + testTempView2, Inner, None, JoinHint.NONE), @@ -275,8 +280,8 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint(hintName, Seq("GlOBal_TeMP.table4", "table5"), table("global_temp", "table4").join(table("global_temp", "table5"))), Join( - ResolvedHint(testRelation4, HintInfo(strategy = Some(st))), - ResolvedHint(testRelation5, HintInfo(strategy = Some(st))), + ResolvedHint(testTempView4, HintInfo(strategy = Some(st))), + ResolvedHint(testTempView5, HintInfo(strategy = Some(st))), Inner, None, JoinHint.NONE), @@ -286,8 +291,8 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint(hintName, Seq("global_temp.TaBlE4", "table5"), table("global_temp", "TaBlE4").join(table("global_temp", "TaBlE5"))), Join( - ResolvedHint(testRelation4, HintInfo(strategy = Some(st))), - testRelation5, + ResolvedHint(testTempView4, HintInfo(strategy = Some(st))), + testTempView5, Inner, None, JoinHint.NONE), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 5cff04117ab69..50f353edb059c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -301,8 +301,8 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { val tempTable2 = Range(1, 20, 2, 10) catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) catalog.createTempView("tbl2", tempTable2, overrideIfExists = false) - assert(catalog.getTempView("tbl1") == Option(tempTable1)) - assert(catalog.getTempView("tbl2") == Option(tempTable2)) + assert(catalog.getTempView("tbl1") == Option(View(None, isTempView = true, tempTable1))) + assert(catalog.getTempView("tbl2") == Option(View(None, isTempView = true, tempTable2))) assert(catalog.getTempView("tbl3").isEmpty) // Temporary view already exists intercept[TempTableAlreadyExistsException] { @@ -310,7 +310,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { } // Temporary view already exists but we override it catalog.createTempView("tbl1", tempTable2, overrideIfExists = true) - assert(catalog.getTempView("tbl1") == Option(tempTable2)) + assert(catalog.getTempView("tbl1") == Option(View(None, isTempView = true, tempTable2))) } } @@ -352,7 +352,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { val tempTable = Range(1, 10, 2, 10) catalog.createTempView("tbl1", tempTable, overrideIfExists = false) catalog.setCurrentDatabase("db2") - assert(catalog.getTempView("tbl1") == Some(tempTable)) + assert(catalog.getTempView("tbl1") == Some(View(None, isTempView = true, tempTable))) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be dropped first catalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false) @@ -366,7 +366,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false) catalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false, purge = false) - assert(catalog.getTempView("tbl1") == Some(tempTable)) + assert(catalog.getTempView("tbl1") == Some(View(None, isTempView = true, tempTable))) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl2")) } } @@ -419,16 +419,16 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { val tempTable = Range(1, 10, 2, 10) catalog.createTempView("tbl1", tempTable, overrideIfExists = false) catalog.setCurrentDatabase("db2") - assert(catalog.getTempView("tbl1") == Option(tempTable)) + assert(catalog.getTempView("tbl1") == Option(View(None, isTempView = true, tempTable))) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be renamed first catalog.renameTable(TableIdentifier("tbl1"), TableIdentifier("tbl3")) assert(catalog.getTempView("tbl1").isEmpty) - assert(catalog.getTempView("tbl3") == Option(tempTable)) + assert(catalog.getTempView("tbl3") == Option(View(None, isTempView = true, tempTable))) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is specified, temp tables are never renamed catalog.renameTable(TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4")) - assert(catalog.getTempView("tbl3") == Option(tempTable)) + assert(catalog.getTempView("tbl3") == Option(View(None, isTempView = true, tempTable))) assert(catalog.getTempView("tbl4").isEmpty) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4")) } @@ -626,7 +626,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { .asInstanceOf[UnresolvedCatalogRelation].tableMeta == metastoreTable1) // Otherwise, we'll first look up a temporary table with the same name assert(catalog.lookupRelation(TableIdentifier("tbl1")) - == SubqueryAlias("tbl1", tempTable1)) + == SubqueryAlias("tbl1", View(None, isTempView = true, tempTable1))) // Then, if that does not exist, look up the relation in the current database catalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false) assert(catalog.lookupRelation(TableIdentifier("tbl1")).children.head @@ -1581,11 +1581,11 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { original.copyStateTo(clone) assert(original ne clone) - assert(clone.getTempView("copytest1") == Some(tempTable1)) + assert(clone.getTempView("copytest1") == Some(View(None, isTempView = true, tempTable1))) // check if clone and original independent clone.dropTable(TableIdentifier("copytest1"), ignoreIfNotExists = false, purge = false) - assert(original.getTempView("copytest1") == Some(tempTable1)) + assert(original.getTempView("copytest1") == Some(View(None, isTempView = true, tempTable1))) val tempTable2 = Range(1, 20, 2, 10) original.createTempView("copytest2", tempTable2, overrideIfExists = false) From e36fc34d37535182115e2773ee137df688ad6957 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 22 Feb 2021 19:51:40 -0800 Subject: [PATCH 26/30] Revert View signature change --- .../sql/catalyst/analysis/Analyzer.scala | 26 ++-- .../sql/catalyst/catalog/SessionCatalog.scala | 17 ++- .../sql/catalyst/catalog/interface.scala | 13 +- .../plans/logical/basicLogicalOperators.scala | 15 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 37 +++-- .../sql/catalyst/analysis/AnalysisTest.scala | 11 ++ .../catalyst/analysis/ResolveHintsSuite.scala | 138 +++++++++--------- .../catalog/SessionCatalogSuite.scala | 39 +++-- .../spark/sql/execution/command/views.scala | 46 ++++-- .../spark/sql/internal/CatalogImpl.scala | 2 +- 10 files changed, 202 insertions(+), 142 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d00c07fffe5bb..f2b14d8233ebf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -871,7 +871,10 @@ class Analyzer(override val catalogManager: CatalogManager) } private def getTempViewRawPlan(plan: LogicalPlan): LogicalPlan = { - EliminateDataFrameTempViews(EliminateSubqueryAliases(plan)) + EliminateSubqueryAliases(plan) match { + case v: View if v.isDataFrameTempView => v.child + case other => other + } } /** @@ -1112,7 +1115,7 @@ class Analyzer(override val catalogManager: CatalogManager) // The view's child should be a logical plan parsed from the `desc.viewText`, the variable // `viewText` should be defined, or else we throw an error on the generation of the View // operator. - case view @ View(Some(desc), isTempView, child) if !child.resolved => + case view @ View(desc, isTempView, child) if !child.resolved => // Resolve all the UnresolvedRelations and Views in the child. val newChild = AnalysisContext.withAnalysisContext(desc) { val nestedViewDepth = AnalysisContext.get.nestedViewDepth @@ -1143,8 +1146,8 @@ class Analyzer(override val catalogManager: CatalogManager) // (e.g., spark.read.parquet("path").createOrReplaceTempView("t"). // Thus, we need to look at the raw plan of a temporary view. getTempViewRawPlan(relation) match { - case v: View if v.desc.isDefined => - throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.get.identifier, table) + case v: View => + throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.identifier, table) case other => i.copy(table = other) } @@ -1169,9 +1172,9 @@ class Analyzer(override val catalogManager: CatalogManager) lookupRelation(u.multipartIdentifier, u.options, false) .map(EliminateSubqueryAliases(_)) .map { - case v: View if v.desc.isDefined => + case v: View => throw QueryCompilationErrors.writeIntoViewNotAllowedError( - v.desc.get.identifier, write) + v.desc.identifier, write) case u: UnresolvedCatalogRelation => throw QueryCompilationErrors.writeIntoV1TableNotAllowedError( u.tableMeta.identifier, write) @@ -3735,17 +3738,6 @@ object EliminateSubqueryAliases extends Rule[LogicalPlan] { } } -/** - * Removes [[View]] operators from the plan if they are temp views created from dataframes. - */ -object EliminateDataFrameTempViews extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = AnalysisHelper.allowInvokingTransformsInAnalyzer { - plan transformUp { - case v: View if v.isDataFrameTempView => v.child - } - } -} - /** * Removes [[Union]] operators from the plan if it just has one child. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 191d7e6a9da3e..74a80f566f94a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -640,8 +640,7 @@ class SessionCatalog( } /** - * Generate a [[View]] operator from the view description if the view stores sql text, - * otherwise, it is same to `getRawGlobalTempView` + * Generate a [[View]] operator from the global temporary view stored. */ def getGlobalTempView(name: String): Option[LogicalPlan] = { getRawGlobalTempView(name).map(getTempViewPlan) @@ -682,7 +681,7 @@ class SessionCatalog( val table = formatTableName(name.table) if (name.database.isEmpty) { tempViews.get(table).map { - case TemporaryViewRelation(metadata) => metadata + case TemporaryViewRelation(metadata, _) => metadata case plan => CatalogTable( identifier = TableIdentifier(table), @@ -692,7 +691,7 @@ class SessionCatalog( }.getOrElse(getTableMetadata(name)) } else if (formatDatabaseName(name.database.get) == globalTempViewManager.database) { globalTempViewManager.get(table).map { - case TemporaryViewRelation(metadata) => metadata + case TemporaryViewRelation(metadata, _) => metadata case plan => CatalogTable( identifier = TableIdentifier(table, Some(globalTempViewManager.database)), @@ -837,9 +836,11 @@ class SessionCatalog( private def getTempViewPlan(plan: LogicalPlan): LogicalPlan = { plan match { - case viewInfo: TemporaryViewRelation => - fromCatalogTable(viewInfo.tableMeta, isTempView = true) - case v => View(None, isTempView = true, v) + case TemporaryViewRelation(tableMeta, None) => + fromCatalogTable(tableMeta, isTempView = true) + case TemporaryViewRelation(tableMeta, Some(plan)) => + View(desc = tableMeta, isTempView = true, child = plan) + case other => other } } @@ -877,7 +878,7 @@ class SessionCatalog( Alias(UpCast(UnresolvedAttribute.quoted(name), field.dataType), field.name)( explicitMetadata = Some(field.metadata)) } - View(desc = Some(metadata), isTempView = isTempView, child = Project(projectList, parsedPlan)) + View(desc = metadata, isTempView = isTempView, child = Project(projectList, parsedPlan)) } def lookupTempView(table: String): Option[SubqueryAlias] = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 89cb103a7bf73..b6a23214c9084 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -31,6 +31,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.catalog.CatalogTable.VIEW_CREATED_FROM_DATAFRAME import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Cast, ExprId, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils @@ -467,6 +468,8 @@ object CatalogTable { val VIEW_REFERRED_TEMP_VIEW_NAMES = VIEW_PREFIX + "referredTempViewNames" val VIEW_REFERRED_TEMP_FUNCTION_NAMES = VIEW_PREFIX + "referredTempFunctionsNames" + val VIEW_CREATED_FROM_DATAFRAME = VIEW_PREFIX + "createdFromDataFrame" + def splitLargeTableProp( key: String, value: String, @@ -779,9 +782,15 @@ case class UnresolvedCatalogRelation( /** * A wrapper to store the temporary view info, will be kept in `SessionCatalog` - * and will be transformed to `View` during analysis + * and will be transformed to `View` during analysis. If the temporary view was + * created from a dataframe, `plan` is set to the analyzed plan for the view. */ -case class TemporaryViewRelation(tableMeta: CatalogTable) extends LeafNode { +case class TemporaryViewRelation( + tableMeta: CatalogTable, + plan: Option[LogicalPlan] = None) extends LeafNode { + require(plan.isEmpty || + (plan.get.resolved && tableMeta.properties.contains(VIEW_CREATED_FROM_DATAFRAME))) + override lazy val resolved: Boolean = false override def output: Seq[Attribute] = Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 9457bf8ea4257..f75ab4e62a8ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.AliasIdentifier import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} +import org.apache.spark.sql.catalyst.catalog.CatalogTable.VIEW_CREATED_FROM_DATAFRAME import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ @@ -449,23 +450,22 @@ case class InsertIntoDir( * case, the child must be already resolved. * This operator will be removed at the end of analysis stage. * - * @param desc An optional view description(CatalogTable) that provides necessary information to - * resolve the view. + * @param desc A view description(CatalogTable) that provides necessary information to resolve the + * view. * @param isTempView A flag to indicate whether the view is temporary or not. * @param child The logical plan of a view operator. If the view description is available, it should * be a logical plan parsed from the `CatalogTable.viewText`. */ case class View( - desc: Option[CatalogTable], + desc: CatalogTable, isTempView: Boolean, child: LogicalPlan) extends UnaryNode { - require(desc.isDefined || (isTempView && child.resolved)) + require(!isDataFrameTempView || child.resolved) override def output: Seq[Attribute] = child.output override def simpleString(maxFields: Int): String = { - val viewIdent = desc.map(d => s"${d.identifier}, ").getOrElse("") - s"View ($viewIdent${output.mkString("[", ",", "]")})" + s"View (${desc.identifier}, ${output.mkString("[", ",", "]")})" } override def doCanonicalize(): LogicalPlan = child match { @@ -473,7 +473,8 @@ case class View( case _ => child.canonicalized } - def isDataFrameTempView: Boolean = desc.isEmpty && isTempView + def isDataFrameTempView: Boolean = + isTempView && desc.properties.contains(VIEW_CREATED_FROM_DATAFRAME) // When resolving a SQL view, we use an extra Project to add cast and alias to make sure the view // output schema doesn't change even if the table referenced by the view is changed after view diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 08517e324ff0f..73bb70321dd6d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -49,6 +49,17 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class AnalysisSuite extends AnalysisTest with Matchers { import org.apache.spark.sql.catalyst.analysis.TestRelations._ + private def checkAnalysisWithTempViewEliminated( + inputPlan: LogicalPlan, + expectedPlan: LogicalPlan, + caseSensitive: Boolean = true): Unit = { + checkAnalysisWithTransform(inputPlan, expectedPlan, caseSensitive) { plan => + plan transformUp { + case v: View if v.isDataFrameTempView => v.child + } + } + } + test("fail for unresolved plan") { intercept[AnalysisException] { // `testRelation` does not have column `b`. @@ -91,32 +102,30 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("analyze project") { - val testTempView = View(None, isTempView = true, testRelation) - checkAnalysis( Project(Seq(UnresolvedAttribute("a")), testRelation), Project(testRelation.output, testRelation)) - checkAnalysis( + checkAnalysisWithTempViewEliminated( Project(Seq(UnresolvedAttribute("TbL.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), - Project(testTempView.output, testTempView)) + Project(testRelation.output, testRelation)) assertAnalysisError( Project(Seq(UnresolvedAttribute("tBl.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), Seq("cannot resolve")) - checkAnalysis( + checkAnalysisWithTempViewEliminated( Project(Seq(UnresolvedAttribute("TbL.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), - Project(testTempView.output, testTempView), + Project(testRelation.output, testRelation), caseSensitive = false) - checkAnalysis( + checkAnalysisWithTempViewEliminated( Project(Seq(UnresolvedAttribute("tBl.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), - Project(testTempView.output, testTempView), + Project(testRelation.output, testRelation), caseSensitive = false) } @@ -204,14 +213,12 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("resolve relations") { - val testTempView = View(None, isTempView = true, testRelation) - assertAnalysisError(UnresolvedRelation(TableIdentifier("tAbLe")), Seq()) - checkAnalysis(UnresolvedRelation(TableIdentifier("TaBlE")), testTempView) - checkAnalysis( - UnresolvedRelation(TableIdentifier("tAbLe")), testTempView, caseSensitive = false) - checkAnalysis( - UnresolvedRelation(TableIdentifier("TaBlE")), testTempView, caseSensitive = false) + checkAnalysisWithTempViewEliminated(UnresolvedRelation(TableIdentifier("TaBlE")), testRelation) + checkAnalysisWithTempViewEliminated( + UnresolvedRelation(TableIdentifier("tAbLe")), testRelation, caseSensitive = false) + checkAnalysisWithTempViewEliminated( + UnresolvedRelation(TableIdentifier("TaBlE")), testRelation, caseSensitive = false) } test("divide should be casted into fractional types") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 37db4be502a83..4967a15271047 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -59,6 +59,17 @@ trait AnalysisTest extends PlanTest { } } + protected def checkAnalysisWithTransform( + inputPlan: LogicalPlan, + expectedPlan: LogicalPlan, + caseSensitive: Boolean = true)(transform: LogicalPlan => LogicalPlan): Unit = { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val analyzer = getAnalyzer + val actualPlan = analyzer.executeAndCheck(inputPlan, new QueryPlanningTracker) + comparePlans(transform(actualPlan), expectedPlan) + } + } + protected override def comparePlans( plan1: LogicalPlan, plan2: LogicalPlan, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index c1417e5965227..5f45eb85a5387 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -30,120 +30,126 @@ import org.apache.spark.sql.types.IntegerType class ResolveHintsSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.analysis.TestRelations._ - val testTempView = View(None, isTempView = true, testRelation) - val testTempView2 = View(None, isTempView = true, testRelation2) - val testTempView4 = View(None, isTempView = true, testRelation4) - val testTempView5 = View(None, isTempView = true, testRelation5) + private def checkAnalysisWithTempViewEliminated( + inputPlan: LogicalPlan, + expectedPlan: LogicalPlan, + caseSensitive: Boolean = true): Unit = { + checkAnalysisWithTransform(inputPlan, expectedPlan, caseSensitive) { plan => + plan transformUp { + case v: View if v.isDataFrameTempView => v.child + } + } + } test("invalid hints should be ignored") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("some_random_hint_that_does_not_exist", Seq("TaBlE"), table("TaBlE")), - testTempView, + testRelation, caseSensitive = false) } test("case-sensitive or insensitive parameters") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("TaBlE"), table("TaBlE")), - ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("table"), table("TaBlE")), - ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("TaBlE"), table("TaBlE")), - ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = true) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("table"), table("TaBlE")), - testTempView, + testRelation, caseSensitive = true) } test("multiple broadcast hint aliases") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("table", "table2"), table("table").join(table("table2"))), - Join(ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), - ResolvedHint(testTempView2, HintInfo(strategy = Some(BROADCAST))), + Join(ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testRelation2, HintInfo(strategy = Some(BROADCAST))), Inner, None, JoinHint.NONE), caseSensitive = false) } test("do not traverse past existing broadcast hints") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("table"), ResolvedHint(table("table").where('a > 1), HintInfo(strategy = Some(BROADCAST)))), - ResolvedHint(testTempView.where('a > 1), HintInfo(strategy = Some(BROADCAST))).analyze, + ResolvedHint(testRelation.where('a > 1), HintInfo(strategy = Some(BROADCAST))).analyze, caseSensitive = false) } test("should work for subqueries") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").as("tableAlias")), - ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").subquery('tableAlias)), - ResolvedHint(testTempView, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) // Negative case: if the alias doesn't match, don't match the original table name. - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("table"), table("table").as("tableAlias")), - testTempView, + testRelation, caseSensitive = false) } test("do not traverse past subquery alias") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("MAPJOIN", Seq("table"), table("table").where('a > 1).subquery('tableAlias)), - testTempView.where('a > 1).analyze, + testRelation.where('a > 1).analyze, caseSensitive = false) } test("should work for CTE") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( CatalystSqlParser.parsePlan( """ |WITH ctetable AS (SELECT * FROM table WHERE a > 1) |SELECT /*+ BROADCAST(ctetable) */ * FROM ctetable """.stripMargin ), - ResolvedHint(testTempView.where('a > 1).select('a), HintInfo(strategy = Some(BROADCAST))) + ResolvedHint(testRelation.where('a > 1).select('a), HintInfo(strategy = Some(BROADCAST))) .select('a).analyze, caseSensitive = false) } test("should not traverse down CTE") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( CatalystSqlParser.parsePlan( """ |WITH ctetable AS (SELECT * FROM table WHERE a > 1) |SELECT /*+ BROADCAST(table) */ * FROM ctetable """.stripMargin ), - testTempView.where('a > 1).select('a).select('a).analyze, + testRelation.where('a > 1).select('a).select('a).analyze, caseSensitive = false) } test("coalesce and repartition hint") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("COALESCE", Seq(Literal(10)), table("TaBlE")), - Repartition(numPartitions = 10, shuffle = false, child = testTempView)) - checkAnalysis( + Repartition(numPartitions = 10, shuffle = false, child = testRelation)) + checkAnalysisWithTempViewEliminated( UnresolvedHint("coalesce", Seq(Literal(20)), table("TaBlE")), - Repartition(numPartitions = 20, shuffle = false, child = testTempView)) - checkAnalysis( + Repartition(numPartitions = 20, shuffle = false, child = testRelation)) + checkAnalysisWithTempViewEliminated( UnresolvedHint("REPARTITION", Seq(Literal(100)), table("TaBlE")), - Repartition(numPartitions = 100, shuffle = true, child = testTempView)) - checkAnalysis( + Repartition(numPartitions = 100, shuffle = true, child = testRelation)) + checkAnalysisWithTempViewEliminated( UnresolvedHint("RePARTITion", Seq(Literal(200)), table("TaBlE")), - Repartition(numPartitions = 200, shuffle = true, child = testTempView)) + Repartition(numPartitions = 200, shuffle = true, child = testRelation)) val errMsg = "COALESCE Hint expects a partition number as a parameter" @@ -157,18 +163,18 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint("COALESCE", Seq(Literal(1.0)), table("TaBlE")), Seq(errMsg)) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("RePartition", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), - RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testTempView, 10)) + RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testRelation, 10)) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("REPARTITION", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), - RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testTempView, 10)) + RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testRelation, 10)) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("REPARTITION", Seq(UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( - Seq(AttributeReference("a", IntegerType)()), testTempView, None)) + Seq(AttributeReference("a", IntegerType)()), testRelation, None)) val e = intercept[IllegalArgumentException] { checkAnalysis( @@ -176,23 +182,23 @@ class ResolveHintsSuite extends AnalysisTest { Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), table("TaBlE")), RepartitionByExpression( - Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testTempView, 10) + Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testRelation, 10) ) } e.getMessage.contains("For range partitioning use REPARTITION_BY_RANGE instead") - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint( "REPARTITION_BY_RANGE", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( - Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testTempView, 10)) + Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testRelation, 10)) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint( "REPARTITION_BY_RANGE", Seq(UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), - testTempView, None)) + testRelation, None)) val errMsg2 = "REPARTITION Hint parameter should include columns, but" @@ -230,9 +236,9 @@ class ResolveHintsSuite extends AnalysisTest { test("log warnings for invalid hints") { val logAppender = new LogAppender("invalid hints") withLogAppender(logAppender) { - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE")), - testTempView, + testRelation, caseSensitive = false) } assert(logAppender.loggingEvents.exists( @@ -241,9 +247,9 @@ class ResolveHintsSuite extends AnalysisTest { } test("SPARK-30003: Do not throw stack overflow exception in non-root unknown hint resolution") { - checkAnalysis( + checkAnalysisWithTempViewEliminated( Project(testRelation.output, UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE"))), - Project(testTempView.output, testTempView), + Project(testRelation.output, testRelation), caseSensitive = false) } @@ -253,46 +259,46 @@ class ResolveHintsSuite extends AnalysisTest { ("SHUFFLE_HASH", SHUFFLE_HASH), ("SHUFFLE_REPLICATE_NL", SHUFFLE_REPLICATE_NL)).foreach { case (hintName, st) => // local temp table (single-part identifier case) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint(hintName, Seq("table", "table2"), table("TaBlE").join(table("TaBlE2"))), Join( - ResolvedHint(testTempView, HintInfo(strategy = Some(st))), - ResolvedHint(testTempView2, HintInfo(strategy = Some(st))), + ResolvedHint(testRelation, HintInfo(strategy = Some(st))), + ResolvedHint(testRelation2, HintInfo(strategy = Some(st))), Inner, None, JoinHint.NONE), caseSensitive = false) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint(hintName, Seq("TaBlE", "table2"), table("TaBlE").join(table("TaBlE2"))), Join( - ResolvedHint(testTempView, HintInfo(strategy = Some(st))), - testTempView2, + ResolvedHint(testRelation, HintInfo(strategy = Some(st))), + testRelation2, Inner, None, JoinHint.NONE), caseSensitive = true) // global temp table (multi-part identifier case) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint(hintName, Seq("GlOBal_TeMP.table4", "table5"), table("global_temp", "table4").join(table("global_temp", "table5"))), Join( - ResolvedHint(testTempView4, HintInfo(strategy = Some(st))), - ResolvedHint(testTempView5, HintInfo(strategy = Some(st))), + ResolvedHint(testRelation4, HintInfo(strategy = Some(st))), + ResolvedHint(testRelation5, HintInfo(strategy = Some(st))), Inner, None, JoinHint.NONE), caseSensitive = false) - checkAnalysis( + checkAnalysisWithTempViewEliminated( UnresolvedHint(hintName, Seq("global_temp.TaBlE4", "table5"), table("global_temp", "TaBlE4").join(table("global_temp", "TaBlE5"))), Join( - ResolvedHint(testTempView4, HintInfo(strategy = Some(st))), - testTempView5, + ResolvedHint(testRelation4, HintInfo(strategy = Some(st))), + testRelation5, Inner, None, JoinHint.NONE), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 50f353edb059c..635fea9114434 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -22,7 +22,7 @@ import scala.concurrent.duration._ import org.scalatest.concurrent.Eventually import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} +import org.apache.spark.sql.catalyst.{AliasIdentifier, FunctionIdentifier, QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -84,6 +84,12 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.reset() } } + + private def getTempViewRawPlan(plan: Option[LogicalPlan]): Option[LogicalPlan] = plan match { + case Some(v: View) if v.isDataFrameTempView => Some(v.child) + case other => other + } + // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -301,16 +307,16 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { val tempTable2 = Range(1, 20, 2, 10) catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) catalog.createTempView("tbl2", tempTable2, overrideIfExists = false) - assert(catalog.getTempView("tbl1") == Option(View(None, isTempView = true, tempTable1))) - assert(catalog.getTempView("tbl2") == Option(View(None, isTempView = true, tempTable2))) - assert(catalog.getTempView("tbl3").isEmpty) + assert(getTempViewRawPlan(catalog.getTempView("tbl1")) == Option(tempTable1)) + assert(getTempViewRawPlan(catalog.getTempView("tbl2")) == Option(tempTable2)) + assert(getTempViewRawPlan(catalog.getTempView("tbl3")).isEmpty) // Temporary view already exists intercept[TempTableAlreadyExistsException] { catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) } // Temporary view already exists but we override it catalog.createTempView("tbl1", tempTable2, overrideIfExists = true) - assert(catalog.getTempView("tbl1") == Option(View(None, isTempView = true, tempTable2))) + assert(getTempViewRawPlan(catalog.getTempView("tbl1")) == Option(tempTable2)) } } @@ -352,7 +358,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { val tempTable = Range(1, 10, 2, 10) catalog.createTempView("tbl1", tempTable, overrideIfExists = false) catalog.setCurrentDatabase("db2") - assert(catalog.getTempView("tbl1") == Some(View(None, isTempView = true, tempTable))) + assert(getTempViewRawPlan(catalog.getTempView("tbl1")) == Some(tempTable)) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be dropped first catalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false) @@ -366,7 +372,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false) catalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false, purge = false) - assert(catalog.getTempView("tbl1") == Some(View(None, isTempView = true, tempTable))) + assert(getTempViewRawPlan(catalog.getTempView("tbl1")) == Some(tempTable)) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl2")) } } @@ -419,16 +425,16 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { val tempTable = Range(1, 10, 2, 10) catalog.createTempView("tbl1", tempTable, overrideIfExists = false) catalog.setCurrentDatabase("db2") - assert(catalog.getTempView("tbl1") == Option(View(None, isTempView = true, tempTable))) + assert(getTempViewRawPlan(catalog.getTempView("tbl1")) == Option(tempTable)) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be renamed first catalog.renameTable(TableIdentifier("tbl1"), TableIdentifier("tbl3")) assert(catalog.getTempView("tbl1").isEmpty) - assert(catalog.getTempView("tbl3") == Option(View(None, isTempView = true, tempTable))) + assert(getTempViewRawPlan(catalog.getTempView("tbl3")) == Option(tempTable)) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is specified, temp tables are never renamed catalog.renameTable(TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4")) - assert(catalog.getTempView("tbl3") == Option(View(None, isTempView = true, tempTable))) + assert(getTempViewRawPlan(catalog.getTempView("tbl3")) == Option(tempTable)) assert(catalog.getTempView("tbl4").isEmpty) assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4")) } @@ -625,8 +631,9 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { assert(catalog.lookupRelation(TableIdentifier("tbl1", Some("db2"))).children.head .asInstanceOf[UnresolvedCatalogRelation].tableMeta == metastoreTable1) // Otherwise, we'll first look up a temporary table with the same name - assert(catalog.lookupRelation(TableIdentifier("tbl1")) - == SubqueryAlias("tbl1", View(None, isTempView = true, tempTable1))) + val tbl1 = catalog.lookupRelation(TableIdentifier("tbl1")).asInstanceOf[SubqueryAlias] + assert(tbl1.identifier == AliasIdentifier("tbl1")) + assert(getTempViewRawPlan(Some(tbl1.child)).get == tempTable1) // Then, if that does not exist, look up the relation in the current database catalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false) assert(catalog.lookupRelation(TableIdentifier("tbl1")).children.head @@ -654,7 +661,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { // Look up a view. catalog.setCurrentDatabase("default") - val view = View(desc = Some(metadata), isTempView = false, child = getViewPlan(metadata)) + val view = View(desc = metadata, isTempView = false, child = getViewPlan(metadata)) comparePlans(catalog.lookupRelation(TableIdentifier("view1", Some("db3"))), SubqueryAlias(Seq(CatalogManager.SESSION_CATALOG_NAME, "db3", "view1"), view)) // Look up a view using current database of the session catalog. @@ -673,7 +680,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { assert(metadata.viewText.isDefined) assert(metadata.viewCatalogAndNamespace == Seq(CatalogManager.SESSION_CATALOG_NAME, "db2")) - val view = View(desc = Some(metadata), isTempView = false, child = getViewPlan(metadata)) + val view = View(desc = metadata, isTempView = false, child = getViewPlan(metadata)) comparePlans(catalog.lookupRelation(TableIdentifier("view2", Some("db3"))), SubqueryAlias(Seq(CatalogManager.SESSION_CATALOG_NAME, "db3", "view2"), view)) } @@ -1581,11 +1588,11 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { original.copyStateTo(clone) assert(original ne clone) - assert(clone.getTempView("copytest1") == Some(View(None, isTempView = true, tempTable1))) + assert(getTempViewRawPlan(clone.getTempView("copytest1")) == Some(tempTable1)) // check if clone and original independent clone.dropTable(TableIdentifier("copytest1"), ignoreIfNotExists = false, purge = false) - assert(original.getTempView("copytest1") == Some(View(None, isTempView = true, tempTable1))) + assert(getTempViewRawPlan(original.getTempView("copytest1")) == Some(tempTable1)) val tempTable2 = Range(1, 20, 2, 10) original.createTempView("copytest2", tempTable2, overrideIfExists = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 5cea5381aa7ae..8e5bda61ad6aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, Pe import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, SessionCatalog, TemporaryViewRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, SubqueryExpression, UserDefinedExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View, With} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} @@ -115,8 +115,7 @@ case class CreateViewCommand( if (viewType == LocalTempView) { val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) - if (replace && catalog.getRawTempView(name.table).isDefined && - !catalog.getRawTempView(name.table).get.sameResult(aliasedPlan)) { + if (replace && !isSamePlan(catalog.getRawTempView(name.table), aliasedPlan)) { logInfo(s"Try to uncache ${name.quotedString} before replacing.") checkCyclicViewReference(analyzedPlan, Seq(name), name) CommandUtils.uncacheTableOrView(sparkSession, name.quotedString) @@ -131,15 +130,16 @@ case class CreateViewCommand( aliasedPlan.schema, originalText)) } else { - aliasedPlan + TemporaryViewRelation( + prepareTemporaryViewFromDataFrame(name, aliasedPlan), + Some(aliasedPlan)) } catalog.createTempView(name.table, tableDefinition, overrideIfExists = replace) } else if (viewType == GlobalTempView) { val db = sparkSession.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) val viewIdent = TableIdentifier(name.table, Option(db)) val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) - if (replace && catalog.getRawGlobalTempView(name.table).isDefined && - !catalog.getRawGlobalTempView(name.table).get.sameResult(aliasedPlan)) { + if (replace && !isSamePlan(catalog.getRawGlobalTempView(name.table), aliasedPlan)) { logInfo(s"Try to uncache ${viewIdent.quotedString} before replacing.") checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) CommandUtils.uncacheTableOrView(sparkSession, viewIdent.quotedString) @@ -153,7 +153,9 @@ case class CreateViewCommand( aliasedPlan.schema, originalText)) } else { - aliasedPlan + TemporaryViewRelation( + prepareTemporaryViewFromDataFrame(name, aliasedPlan), + Some(aliasedPlan)) } catalog.createGlobalTempView(name.table, tableDefinition, overrideIfExists = replace) } else if (catalog.tableExists(name)) { @@ -190,6 +192,17 @@ case class CreateViewCommand( Seq.empty[Row] } + /** + * Checks if the temp view (the result of getTempViewRawPlan or getRawGlobalTempView) is storing + * the same plan as the given aliased plan. + */ + private def isSamePlan( + rawTempView: Option[LogicalPlan], + aliasedPlan: LogicalPlan): Boolean = rawTempView match { + case Some(TemporaryViewRelation(_, Some(p))) => p.sameResult(aliasedPlan) + case _ => false + } + /** * If `userSpecifiedColumns` is defined, alias the analyzed plan to the user specified columns, * else return the analyzed plan directly. @@ -487,7 +500,7 @@ object ViewHelper { path: Seq[TableIdentifier], viewIdent: TableIdentifier): Unit = { plan match { - case v @ View(Some(desc), _, _) => + case v @ View(desc, _, _) => val ident = desc.identifier val newPath = path :+ ident // If the table identifier equals to the `viewIdent`, current view node is the same with @@ -547,7 +560,6 @@ object ViewHelper { child.flatMap { case s @ SubqueryAlias(_, view: View) if view.isTempView => Seq(s.identifier.qualifier :+ s.identifier.name) - case w: With => w.innerChildren.flatMap(collectTempViews) case plan => plan.expressions.flatMap(_.flatMap { case e: SubqueryExpression => collectTempViews(e.plan) case _ => Seq.empty @@ -557,7 +569,6 @@ object ViewHelper { def collectTempFunctions(child: LogicalPlan): Seq[String] = { child.flatMap { - case w: With => w.innerChildren.flatMap(collectTempFunctions) case plan => plan.expressions.flatMap(_.flatMap { case e: SubqueryExpression => collectTempFunctions(e.plan) @@ -600,4 +611,19 @@ object ViewHelper { viewText = originalText, properties = newProperties) } + + /** + * Returns a [[CatalogTable]] that contains information for the temporary view created + * from a dataframe. + */ + def prepareTemporaryViewFromDataFrame( + viewName: TableIdentifier, + analyzedPlan: LogicalPlan): CatalogTable = { + CatalogTable( + identifier = viewName, + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = analyzedPlan.schema, + properties = Map((VIEW_CREATED_FROM_DATAFRAME, "true"))) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index ca7491d9fc715..8643eb1301d59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -424,7 +424,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { // same way as how a permanent view is handled. This also avoids a potential issue where a // dependent view becomes invalid because of the above while its data is still cached. val viewText = viewDef match { - case View(Some(desc), _, _) => desc.viewText + case View(desc, _, _) => desc.viewText case _ => None } val plan = sparkSession.sessionState.executePlan(viewDef) From b54df8a6ff8b7193373c404f19bda332d0a80b1b Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 22 Feb 2021 19:53:35 -0800 Subject: [PATCH 27/30] reduce churn --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f2b14d8233ebf..4614ba3c755f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1172,9 +1172,8 @@ class Analyzer(override val catalogManager: CatalogManager) lookupRelation(u.multipartIdentifier, u.options, false) .map(EliminateSubqueryAliases(_)) .map { - case v: View => - throw QueryCompilationErrors.writeIntoViewNotAllowedError( - v.desc.identifier, write) + case v: View => throw QueryCompilationErrors.writeIntoViewNotAllowedError( + v.desc.identifier, write) case u: UnresolvedCatalogRelation => throw QueryCompilationErrors.writeIntoV1TableNotAllowedError( u.tableMeta.identifier, write) From c5651b24398961ff59b49b8756f35f5b97eca275 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 22 Feb 2021 19:55:56 -0800 Subject: [PATCH 28/30] reduce churn --- .../main/scala/org/apache/spark/sql/internal/CatalogImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 8643eb1301d59..145daaffd035f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -424,7 +424,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { // same way as how a permanent view is handled. This also avoids a potential issue where a // dependent view becomes invalid because of the above while its data is still cached. val viewText = viewDef match { - case View(desc, _, _) => desc.viewText + case v: View => v.desc.viewText case _ => None } val plan = sparkSession.sessionState.executePlan(viewDef) From d4e958a6b60009da7ef3a4f2ec227b8fd54927db Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 23 Feb 2021 16:31:28 -0800 Subject: [PATCH 29/30] address PR comments --- .../plans/logical/basicLogicalOperators.scala | 6 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 23 ++----- .../sql/catalyst/analysis/AnalysisTest.scala | 11 +++ .../catalyst/analysis/ResolveHintsSuite.scala | 67 ++++++++----------- .../spark/sql/execution/command/views.scala | 12 ++-- 5 files changed, 56 insertions(+), 63 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index f75ab4e62a8ef..1c5f5fc2c60ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -446,8 +446,10 @@ case class InsertIntoDir( /** * A container for holding the view description(CatalogTable) and info whether the view is temporary * or not. If the view description is available, the child should be a logical plan parsed from the - * `CatalogTable.viewText`. Otherwise, the view is a temporary one created from a dataframe; in this - * case, the child must be already resolved. + * `CatalogTable.viewText`. Otherwise, the view is a temporary one created from a dataframe and the + * view description should contain a `VIEW_CREATED_FROM_DATAFRAME` property; in this case, the child + * must be already resolved. + * * This operator will be removed at the end of analysis stage. * * @param desc A view description(CatalogTable) that provides necessary information to resolve the diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 73bb70321dd6d..a3c26ecdaba2a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -49,17 +49,6 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class AnalysisSuite extends AnalysisTest with Matchers { import org.apache.spark.sql.catalyst.analysis.TestRelations._ - private def checkAnalysisWithTempViewEliminated( - inputPlan: LogicalPlan, - expectedPlan: LogicalPlan, - caseSensitive: Boolean = true): Unit = { - checkAnalysisWithTransform(inputPlan, expectedPlan, caseSensitive) { plan => - plan transformUp { - case v: View if v.isDataFrameTempView => v.child - } - } - } - test("fail for unresolved plan") { intercept[AnalysisException] { // `testRelation` does not have column `b`. @@ -106,7 +95,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { Project(Seq(UnresolvedAttribute("a")), testRelation), Project(testRelation.output, testRelation)) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( Project(Seq(UnresolvedAttribute("TbL.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), Project(testRelation.output, testRelation)) @@ -116,13 +105,13 @@ class AnalysisSuite extends AnalysisTest with Matchers { SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), Seq("cannot resolve")) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( Project(Seq(UnresolvedAttribute("TbL.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), Project(testRelation.output, testRelation), caseSensitive = false) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( Project(Seq(UnresolvedAttribute("tBl.a")), SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), Project(testRelation.output, testRelation), @@ -214,10 +203,10 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("resolve relations") { assertAnalysisError(UnresolvedRelation(TableIdentifier("tAbLe")), Seq()) - checkAnalysisWithTempViewEliminated(UnresolvedRelation(TableIdentifier("TaBlE")), testRelation) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper(UnresolvedRelation(TableIdentifier("TaBlE")), testRelation) + checkAnalysisWithoutViewWrapper( UnresolvedRelation(TableIdentifier("tAbLe")), testRelation, caseSensitive = false) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedRelation(TableIdentifier("TaBlE")), testRelation, caseSensitive = false) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 4967a15271047..1546976b5640f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -70,6 +70,17 @@ trait AnalysisTest extends PlanTest { } } + protected def checkAnalysisWithoutViewWrapper( + inputPlan: LogicalPlan, + expectedPlan: LogicalPlan, + caseSensitive: Boolean = true): Unit = { + checkAnalysisWithTransform(inputPlan, expectedPlan, caseSensitive) { plan => + plan transformUp { + case v: View if v.isDataFrameTempView => v.child + } + } + } + protected override def comparePlans( plan1: LogicalPlan, plan2: LogicalPlan, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index 5f45eb85a5387..9db64c684c40f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -30,48 +30,37 @@ import org.apache.spark.sql.types.IntegerType class ResolveHintsSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.analysis.TestRelations._ - private def checkAnalysisWithTempViewEliminated( - inputPlan: LogicalPlan, - expectedPlan: LogicalPlan, - caseSensitive: Boolean = true): Unit = { - checkAnalysisWithTransform(inputPlan, expectedPlan, caseSensitive) { plan => - plan transformUp { - case v: View if v.isDataFrameTempView => v.child - } - } - } - test("invalid hints should be ignored") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("some_random_hint_that_does_not_exist", Seq("TaBlE"), table("TaBlE")), testRelation, caseSensitive = false) } test("case-sensitive or insensitive parameters") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("TaBlE"), table("TaBlE")), ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("table"), table("TaBlE")), ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("TaBlE"), table("TaBlE")), ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = true) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("table"), table("TaBlE")), testRelation, caseSensitive = true) } test("multiple broadcast hint aliases") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("table", "table2"), table("table").join(table("table2"))), Join(ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), ResolvedHint(testRelation2, HintInfo(strategy = Some(BROADCAST))), @@ -80,7 +69,7 @@ class ResolveHintsSuite extends AnalysisTest { } test("do not traverse past existing broadcast hints") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("table"), ResolvedHint(table("table").where('a > 1), HintInfo(strategy = Some(BROADCAST)))), ResolvedHint(testRelation.where('a > 1), HintInfo(strategy = Some(BROADCAST))).analyze, @@ -88,32 +77,32 @@ class ResolveHintsSuite extends AnalysisTest { } test("should work for subqueries") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").as("tableAlias")), ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").subquery('tableAlias)), ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) // Negative case: if the alias doesn't match, don't match the original table name. - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("table"), table("table").as("tableAlias")), testRelation, caseSensitive = false) } test("do not traverse past subquery alias") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("table"), table("table").where('a > 1).subquery('tableAlias)), testRelation.where('a > 1).analyze, caseSensitive = false) } test("should work for CTE") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( CatalystSqlParser.parsePlan( """ |WITH ctetable AS (SELECT * FROM table WHERE a > 1) @@ -126,7 +115,7 @@ class ResolveHintsSuite extends AnalysisTest { } test("should not traverse down CTE") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( CatalystSqlParser.parsePlan( """ |WITH ctetable AS (SELECT * FROM table WHERE a > 1) @@ -138,16 +127,16 @@ class ResolveHintsSuite extends AnalysisTest { } test("coalesce and repartition hint") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("COALESCE", Seq(Literal(10)), table("TaBlE")), Repartition(numPartitions = 10, shuffle = false, child = testRelation)) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("coalesce", Seq(Literal(20)), table("TaBlE")), Repartition(numPartitions = 20, shuffle = false, child = testRelation)) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("REPARTITION", Seq(Literal(100)), table("TaBlE")), Repartition(numPartitions = 100, shuffle = true, child = testRelation)) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("RePARTITion", Seq(Literal(200)), table("TaBlE")), Repartition(numPartitions = 200, shuffle = true, child = testRelation)) @@ -163,15 +152,15 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint("COALESCE", Seq(Literal(1.0)), table("TaBlE")), Seq(errMsg)) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("RePartition", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testRelation, 10)) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("REPARTITION", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testRelation, 10)) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("REPARTITION", Seq(UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( Seq(AttributeReference("a", IntegerType)()), testRelation, None)) @@ -187,13 +176,13 @@ class ResolveHintsSuite extends AnalysisTest { } e.getMessage.contains("For range partitioning use REPARTITION_BY_RANGE instead") - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint( "REPARTITION_BY_RANGE", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testRelation, 10)) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint( "REPARTITION_BY_RANGE", Seq(UnresolvedAttribute("a")), table("TaBlE")), RepartitionByExpression( @@ -236,7 +225,7 @@ class ResolveHintsSuite extends AnalysisTest { test("log warnings for invalid hints") { val logAppender = new LogAppender("invalid hints") withLogAppender(logAppender) { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE")), testRelation, caseSensitive = false) @@ -247,7 +236,7 @@ class ResolveHintsSuite extends AnalysisTest { } test("SPARK-30003: Do not throw stack overflow exception in non-root unknown hint resolution") { - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( Project(testRelation.output, UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE"))), Project(testRelation.output, testRelation), caseSensitive = false) @@ -259,7 +248,7 @@ class ResolveHintsSuite extends AnalysisTest { ("SHUFFLE_HASH", SHUFFLE_HASH), ("SHUFFLE_REPLICATE_NL", SHUFFLE_REPLICATE_NL)).foreach { case (hintName, st) => // local temp table (single-part identifier case) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint(hintName, Seq("table", "table2"), table("TaBlE").join(table("TaBlE2"))), Join( @@ -270,7 +259,7 @@ class ResolveHintsSuite extends AnalysisTest { JoinHint.NONE), caseSensitive = false) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint(hintName, Seq("TaBlE", "table2"), table("TaBlE").join(table("TaBlE2"))), Join( @@ -282,7 +271,7 @@ class ResolveHintsSuite extends AnalysisTest { caseSensitive = true) // global temp table (multi-part identifier case) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint(hintName, Seq("GlOBal_TeMP.table4", "table5"), table("global_temp", "table4").join(table("global_temp", "table5"))), Join( @@ -293,7 +282,7 @@ class ResolveHintsSuite extends AnalysisTest { JoinHint.NONE), caseSensitive = false) - checkAnalysisWithTempViewEliminated( + checkAnalysisWithoutViewWrapper( UnresolvedHint(hintName, Seq("global_temp.TaBlE4", "table5"), table("global_temp", "TaBlE4").join(table("global_temp", "TaBlE5"))), Join( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 8e5bda61ad6aa..bae7c54f0b99d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, Pe import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, SessionCatalog, TemporaryViewRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, SubqueryExpression, UserDefinedExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} @@ -200,6 +200,7 @@ case class CreateViewCommand( rawTempView: Option[LogicalPlan], aliasedPlan: LogicalPlan): Boolean = rawTempView match { case Some(TemporaryViewRelation(_, Some(p))) => p.sameResult(aliasedPlan) + case Some(p) => p.sameResult(aliasedPlan) case _ => false } @@ -500,8 +501,8 @@ object ViewHelper { path: Seq[TableIdentifier], viewIdent: TableIdentifier): Unit = { plan match { - case v @ View(desc, _, _) => - val ident = desc.identifier + case v: View => + val ident = v.desc.identifier val newPath = path :+ ident // If the table identifier equals to the `viewIdent`, current view node is the same with // the altered view. We detect a view reference cycle, should throw an AnalysisException. @@ -558,8 +559,9 @@ object ViewHelper { catalog: SessionCatalog, child: LogicalPlan): (Seq[Seq[String]], Seq[String]) = { def collectTempViews(child: LogicalPlan): Seq[Seq[String]] = { child.flatMap { - case s @ SubqueryAlias(_, view: View) if view.isTempView => - Seq(s.identifier.qualifier :+ s.identifier.name) + case view: View if view.isTempView => + val ident = view.desc.identifier + Seq(ident.database.toSeq :+ ident.table) case plan => plan.expressions.flatMap(_.flatMap { case e: SubqueryExpression => collectTempViews(e.plan) case _ => Seq.empty From 82d58baa98235f912141b2b40cdcd3bf0bdbe744 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 23 Feb 2021 20:36:54 -0800 Subject: [PATCH 30/30] Address PR comments --- .../spark/sql/catalyst/analysis/Analyzer.scala | 8 ++++---- .../plans/logical/basicLogicalOperators.scala | 2 +- .../sql/catalyst/analysis/AnalysisTest.scala | 17 ++++------------- 3 files changed, 9 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 214958981760e..2cec2cc5c3706 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -869,7 +869,7 @@ class Analyzer(override val catalogManager: CatalogManager) } } - private def getTempViewRawPlan(plan: LogicalPlan): LogicalPlan = { + private def unwrapRelationPlan(plan: LogicalPlan): LogicalPlan = { EliminateSubqueryAliases(plan) match { case v: View if v.isDataFrameTempView => v.child case other => other @@ -900,7 +900,7 @@ class Analyzer(override val catalogManager: CatalogManager) case write: V2WriteCommand => write.table match { case UnresolvedRelation(ident, _, false) => - lookupTempView(ident, performCheck = true).map(getTempViewRawPlan).map { + lookupTempView(ident, performCheck = true).map(unwrapRelationPlan).map { case r: DataSourceV2Relation => write.withNewTable(r) case _ => throw QueryCompilationErrors.writeIntoTempViewNotAllowedError(ident.quoted) }.getOrElse(write) @@ -1154,8 +1154,8 @@ class Analyzer(override val catalogManager: CatalogManager) // Inserting into a file-based temporary view is allowed. // (e.g., spark.read.parquet("path").createOrReplaceTempView("t"). - // Thus, we need to look at the raw plan of a temporary view. - getTempViewRawPlan(relation) match { + // Thus, we need to look at the raw plan if `relation` is a temporary view. + unwrapRelationPlan(relation) match { case v: View => throw QueryCompilationErrors.insertIntoViewNotAllowedError(v.desc.identifier, table) case other => i.copy(table = other) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 1c5f5fc2c60ff..3f20d8f67b44d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -445,7 +445,7 @@ case class InsertIntoDir( /** * A container for holding the view description(CatalogTable) and info whether the view is temporary - * or not. If the view description is available, the child should be a logical plan parsed from the + * or not. If it's a SQL (temp) view, the child should be a logical plan parsed from the * `CatalogTable.viewText`. Otherwise, the view is a temporary one created from a dataframe and the * view description should contain a `VIEW_CREATED_FROM_DATAFRAME` property; in this case, the child * must be already resolved. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 1546976b5640f..7248424a68ad9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -59,25 +59,16 @@ trait AnalysisTest extends PlanTest { } } - protected def checkAnalysisWithTransform( - inputPlan: LogicalPlan, - expectedPlan: LogicalPlan, - caseSensitive: Boolean = true)(transform: LogicalPlan => LogicalPlan): Unit = { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - val analyzer = getAnalyzer - val actualPlan = analyzer.executeAndCheck(inputPlan, new QueryPlanningTracker) - comparePlans(transform(actualPlan), expectedPlan) - } - } - protected def checkAnalysisWithoutViewWrapper( inputPlan: LogicalPlan, expectedPlan: LogicalPlan, caseSensitive: Boolean = true): Unit = { - checkAnalysisWithTransform(inputPlan, expectedPlan, caseSensitive) { plan => - plan transformUp { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val actualPlan = getAnalyzer.executeAndCheck(inputPlan, new QueryPlanningTracker) + val transformed = actualPlan transformUp { case v: View if v.isDataFrameTempView => v.child } + comparePlans(transformed, expectedPlan) } }