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 12f9a61fc2b6..c80d9d2d7c18 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 @@ -1344,6 +1344,10 @@ class SessionCatalog( !hiveFunctions.contains(name.funcName.toLowerCase(Locale.ROOT)) } + def isTempFunction(name: String): Boolean = { + isTemporaryFunction(FunctionIdentifier(name)) + } + /** * Return whether this function has been registered in the function registry of the current * session. If not existed, return false. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 080ddf1d027e..b0b9d7b15cef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -94,6 +94,10 @@ private[sql] trait LookupCatalog extends Logging { * Extract catalog and identifier from a multi-part name with the current catalog if needed. * Catalog name takes precedence over identifier, but for a single-part name, identifier takes * precedence over catalog name. + * + * Note that, this pattern is used to look up permanent catalog objects like table, view, + * function, etc. If you need to look up temp objects like temp view, please do it separately + * before calling this pattern, as temp objects don't belong to any catalog. */ object CatalogAndIdentifier { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper @@ -103,16 +107,7 @@ private[sql] trait LookupCatalog extends Logging { def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Identifier)] = { assert(nameParts.nonEmpty) if (nameParts.length == 1) { - // If the current catalog is session catalog, the current namespace is not used because - // the single-part name could be referencing a temp view, which doesn't belong to any - // namespaces. An empty namespace will be resolved inside the session catalog - // implementation when a relation is looked up. - val ns = if (CatalogV2Util.isSessionCatalog(currentCatalog)) { - Array.empty[String] - } else { - catalogManager.currentNamespace - } - Some((currentCatalog, Identifier.of(ns, nameParts.head))) + Some((currentCatalog, Identifier.of(catalogManager.currentNamespace, nameParts.head))) } else if (nameParts.head.equalsIgnoreCase(globalTempDB)) { // Conceptually global temp views are in a special reserved catalog. However, the v2 catalog // API does not support view yet, and we have to use v1 commands to deal with global temp diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index b2f27e4740cb..b36ded3d9d2a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -50,6 +50,7 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { }) when(manager.currentCatalog).thenReturn(sessionCatalog) when(manager.v2SessionCatalog).thenReturn(sessionCatalog) + when(manager.currentNamespace).thenReturn(Array.empty[String]) manager } 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 77d549c28aae..62ebb1c1492a 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 @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} @@ -39,7 +39,8 @@ import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBui class ResolveSessionCatalog( val catalogManager: CatalogManager, conf: SQLConf, - isView: Seq[String] => Boolean) + isTempView: Seq[String] => Boolean, + isTempFunction: String => Boolean) extends Rule[LogicalPlan] with LookupCatalog { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.CatalogV2Util._ @@ -215,8 +216,9 @@ class ResolveSessionCatalog( } AlterDatabaseSetLocationCommand(ns.head, location) - case RenameTableStatement(SessionCatalogAndTable(_, oldName), newNameParts, isView) => - AlterTableRenameCommand(oldName.asTableIdentifier, newNameParts.asTableIdentifier, isView) + // v1 RENAME TABLE supports temp view. + case RenameTableStatement(TempViewOrV1Table(oldName), newName, isView) => + AlterTableRenameCommand(oldName.asTableIdentifier, newName.asTableIdentifier, isView) case DescribeRelation(ResolvedTable(_, ident, _: V1Table), partitionSpec, isExtended) => DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) @@ -228,10 +230,12 @@ class ResolveSessionCatalog( case DescribeColumnStatement( SessionCatalogAndTable(catalog, tbl), colNameParts, isExtended) => loadTable(catalog, tbl.asIdentifier).collect { + // `V1Table` also includes permanent views. case v1Table: V1Table => DescribeColumnCommand(tbl.asTableIdentifier, colNameParts, isExtended) }.getOrElse { - if (isView(tbl)) { + if (isTempView(tbl)) { + // v1 DESCRIBE COLUMN supports temp view. DescribeColumnCommand(tbl.asTableIdentifier, colNameParts, isExtended) } else { throw new AnalysisException("Describing columns is not supported for v2 tables.") @@ -279,8 +283,9 @@ class ResolveSessionCatalog( ignoreIfExists = c.ifNotExists) } - case RefreshTableStatement(SessionCatalogAndTable(_, tbl)) => - RefreshTable(tbl.asTableIdentifier) + // v1 REFRESH TABLE supports temp view. + case RefreshTableStatement(TempViewOrV1Table(name)) => + RefreshTable(name.asTableIdentifier) // For REPLACE TABLE [AS SELECT], we should fail if the catalog is resolved to the // session catalog and the table provider is not v2. @@ -315,11 +320,13 @@ class ResolveSessionCatalog( orCreate = c.orCreate) } - case DropTableStatement(SessionCatalogAndTable(catalog, tbl), ifExists, purge) => - DropTableCommand(tbl.asTableIdentifier, ifExists, isView = false, purge = purge) + // v1 DROP TABLE supports temp view. + case DropTableStatement(TempViewOrV1Table(name), ifExists, purge) => + DropTableCommand(name.asTableIdentifier, ifExists, isView = false, purge = purge) - case DropViewStatement(SessionCatalogAndTable(catalog, viewName), ifExists) => - DropTableCommand(viewName.asTableIdentifier, ifExists, isView = true, purge = false) + // v1 DROP TABLE supports temp view. + case DropViewStatement(TempViewOrV1Table(name), ifExists) => + DropTableCommand(name.asTableIdentifier, ifExists, isView = true, purge = false) case c @ CreateNamespaceStatement(CatalogAndNamespace(catalog, ns), _, _) if isSessionCatalog(catalog) => @@ -393,12 +400,18 @@ class ResolveSessionCatalog( ShowCreateTableAsSerdeCommand(v1TableName.asTableIdentifier) case CacheTableStatement(tbl, plan, isLazy, options) => - val v1TableName = parseV1Table(tbl, "CACHE TABLE") - CacheTableCommand(v1TableName.asTableIdentifier, plan, isLazy, options) + val name = if (plan.isDefined) { + // CACHE TABLE ... AS SELECT creates a temp view with the input query. + // Temp view doesn't belong to any catalog and we shouldn't resolve catalog in the name. + tbl + } else { + parseTempViewOrV1Table(tbl, "CACHE TABLE") + } + CacheTableCommand(name.asTableIdentifier, plan, isLazy, options) case UncacheTableStatement(tbl, ifExists) => - val v1TableName = parseV1Table(tbl, "UNCACHE TABLE") - UncacheTableCommand(v1TableName.asTableIdentifier, ifExists) + val name = parseTempViewOrV1Table(tbl, "UNCACHE TABLE") + UncacheTableCommand(name.asTableIdentifier, ifExists) case TruncateTableStatement(tbl, partitionSpec) => val v1TableName = parseV1Table(tbl, "TRUNCATE TABLE") @@ -427,10 +440,6 @@ class ResolveSessionCatalog( throw new AnalysisException( s"Namespace name should have only one part if specified: ${ns.get.quoted}") } - if (tbl.length > 2) { - throw new AnalysisException( - s"Table name should have at most two parts: ${tbl.quoted}") - } ShowColumnsCommand(db, v1TableName) case AlterTableRecoverPartitionsStatement(tbl) => @@ -470,10 +479,10 @@ class ResolveSessionCatalog( serdeProperties, partitionSpec) - case AlterViewAsStatement(tbl, originalText, query) => - val v1TableName = parseV1Table(tbl, "ALTER VIEW QUERY") + case AlterViewAsStatement(name, originalText, query) => + val viewName = parseTempViewOrV1Table(name, "ALTER VIEW QUERY") AlterViewAsCommand( - v1TableName.asTableIdentifier, + viewName.asTableIdentifier, originalText, query) @@ -481,7 +490,12 @@ class ResolveSessionCatalog( tbl, userSpecifiedColumns, comment, properties, originalText, child, allowExisting, replace, viewType) => - val v1TableName = parseV1Table(tbl, "CREATE VIEW") + val v1TableName = if (viewType != PersistedView) { + // temp view doesn't belong to any catalog and we shouldn't resolve catalog in the name. + tbl + } else { + parseV1Table(tbl, "CREATE VIEW") + } CreateViewCommand( v1TableName.asTableIdentifier, userSpecifiedColumns, @@ -496,56 +510,94 @@ class ResolveSessionCatalog( case ShowTableProperties(r: ResolvedTable, propertyKey) if isSessionCatalog(r.catalog) => ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey) - case DescribeFunctionStatement(CatalogAndIdentifier(catalog, ident), extended) => + case DescribeFunctionStatement(nameParts, extended) => val functionIdent = - parseSessionCatalogFunctionIdentifier("DESCRIBE FUNCTION", catalog, ident) + parseSessionCatalogFunctionIdentifier(nameParts, "DESCRIBE FUNCTION") DescribeFunctionCommand(functionIdent, extended) case ShowFunctionsStatement(userScope, systemScope, pattern, fun) => val (database, function) = fun match { - case Some(CatalogAndIdentifier(catalog, ident)) => + case Some(nameParts) => val FunctionIdentifier(fn, db) = - parseSessionCatalogFunctionIdentifier("SHOW FUNCTIONS", catalog, ident) + parseSessionCatalogFunctionIdentifier(nameParts, "SHOW FUNCTIONS") (db, Some(fn)) case None => (None, pattern) } ShowFunctionsCommand(database, function, userScope, systemScope) - case DropFunctionStatement(CatalogAndIdentifier(catalog, ident), ifExists, isTemp) => + case DropFunctionStatement(nameParts, ifExists, isTemp) => val FunctionIdentifier(function, database) = - parseSessionCatalogFunctionIdentifier("DROP FUNCTION", catalog, ident) + parseSessionCatalogFunctionIdentifier(nameParts, "DROP FUNCTION") DropFunctionCommand(database, function, ifExists, isTemp) - case CreateFunctionStatement(CatalogAndIdentifier(catalog, ident), + case CreateFunctionStatement(nameParts, className, resources, isTemp, ignoreIfExists, replace) => - val FunctionIdentifier(function, database) = - parseSessionCatalogFunctionIdentifier("CREATE FUNCTION", catalog, ident) - CreateFunctionCommand(database, function, className, resources, isTemp, ignoreIfExists, - replace) + if (isTemp) { + // temp func doesn't belong to any catalog and we shouldn't resolve catalog in the name. + val database = if (nameParts.length > 2) { + throw new AnalysisException(s"Unsupported function name '${nameParts.quoted}'") + } else if (nameParts.length == 2) { + Some(nameParts.head) + } else { + None + } + CreateFunctionCommand( + database, + nameParts.last, + className, + resources, + isTemp, + ignoreIfExists, + replace) + } else { + val FunctionIdentifier(function, database) = + parseSessionCatalogFunctionIdentifier(nameParts, "CREATE FUNCTION") + CreateFunctionCommand(database, function, className, resources, isTemp, ignoreIfExists, + replace) + } } + // TODO: move function related v2 statements to the new framework. private def parseSessionCatalogFunctionIdentifier( - sql: String, - catalog: CatalogPlugin, - functionIdent: Identifier): FunctionIdentifier = { - if (isSessionCatalog(catalog)) { - functionIdent.asMultipartIdentifier match { - case Seq(db, fn) => FunctionIdentifier(fn, Some(db)) - case Seq(fn) => FunctionIdentifier(fn, None) - case _ => - throw new AnalysisException(s"Unsupported function name '${functionIdent.quoted}'") - } - } else { - throw new AnalysisException(s"$sql is only supported in v1 catalog") + nameParts: Seq[String], + sql: String): FunctionIdentifier = { + if (nameParts.length == 1 && isTempFunction(nameParts.head)) { + return FunctionIdentifier(nameParts.head) } - } - private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { - val CatalogAndIdentifier(catalog, ident) = tableName - if (!isSessionCatalog(catalog)) { - throw new AnalysisException(s"$sql is only supported with v1 tables.") + nameParts match { + case SessionCatalogAndIdentifier(_, ident) => + if (nameParts.length == 1) { + // If there is only one name part, it means the current catalog is the session catalog. + // Here we don't fill the default database, to keep the error message unchanged for + // v1 commands. + FunctionIdentifier(nameParts.head, None) + } else { + ident.namespace match { + // For name parts like `spark_catalog.t`, we need to fill in the default database so + // that the caller side won't treat it as a temp function. + case Array() if nameParts.head == CatalogManager.SESSION_CATALOG_NAME => + FunctionIdentifier( + ident.name, Some(catalogManager.v1SessionCatalog.getCurrentDatabase)) + case Array(db) => FunctionIdentifier(ident.name, Some(db)) + case _ => + throw new AnalysisException(s"Unsupported function name '$ident'") + } + } + + case _ => throw new AnalysisException(s"$sql is only supported in v1 catalog") } - ident.asMultipartIdentifier + } + + private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = tableName match { + case SessionCatalogAndTable(_, tbl) => tbl + case _ => throw new AnalysisException(s"$sql is only supported with v1 tables.") + } + + private def parseTempViewOrV1Table( + nameParts: Seq[String], sql: String): Seq[String] = nameParts match { + case TempViewOrV1Table(name) => name + case _ => throw new AnalysisException(s"$sql is only supported with temp views or v1 tables.") } private def buildCatalogTable( @@ -584,7 +636,29 @@ class ResolveSessionCatalog( object SessionCatalogAndTable { def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { case SessionCatalogAndIdentifier(catalog, ident) => - Some(catalog -> ident.asMultipartIdentifier) + if (nameParts.length == 1) { + // If there is only one name part, it means the current catalog is the session catalog. + // Here we return the original name part, to keep the error message unchanged for + // v1 commands. + Some(catalog -> nameParts) + } else { + Some(catalog -> ident.asMultipartIdentifier) + } + case _ => None + } + } + + object TempViewOrV1Table { + def unapply(nameParts: Seq[String]): Option[Seq[String]] = nameParts match { + case _ if isTempView(nameParts) => Some(nameParts) + case SessionCatalogAndTable(_, tbl) => + if (nameParts.head == CatalogManager.SESSION_CATALOG_NAME && tbl.length == 1) { + // For name parts like `spark_catalog.t`, we need to fill in the default database so + // that the caller side won't treat it as a temp view. + Some(Seq(catalogManager.v1SessionCatalog.getCurrentDatabase, tbl.head)) + } else { + Some(tbl) + } case _ => None } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index eb658e2d8850..2137fe2d0a8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -174,7 +174,8 @@ abstract class BaseSessionStateBuilder( new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: - new ResolveSessionCatalog(catalogManager, conf, catalog.isView) +: + new ResolveSessionCatalog( + catalogManager, conf, catalog.isTempView, catalog.isTempFunction) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 697e006544ac..a7ef828e4d5d 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -520,7 +520,7 @@ struct -- !query output == Physical Plan == Execute DescribeTableCommand - +- DescribeTableCommand `t`, false + +- DescribeTableCommand `default`.`t`, false -- !query @@ -530,7 +530,7 @@ struct -- !query output == Physical Plan == Execute DescribeTableCommand - +- DescribeTableCommand `t`, true + +- DescribeTableCommand `default`.`t`, true -- !query @@ -544,14 +544,14 @@ struct == Analyzed Logical Plan == col_name: string, data_type: string, comment: string -DescribeTableCommand `t`, false +DescribeTableCommand `default`.`t`, false == Optimized Logical Plan == -DescribeTableCommand `t`, false +DescribeTableCommand `default`.`t`, false == Physical Plan == Execute DescribeTableCommand - +- DescribeTableCommand `t`, false + +- DescribeTableCommand `default`.`t`, false -- !query @@ -571,7 +571,7 @@ struct -- !query output == Physical Plan == Execute DescribeTableCommand - +- DescribeTableCommand `t`, Map(c -> Us, d -> 2), false + +- DescribeTableCommand `default`.`t`, Map(c -> Us, d -> 2), false -- !query 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 eabcb81c5064..2acd7e39fe94 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 @@ -1859,12 +1859,12 @@ class DataSourceV2SQLSuite withTable(t) { spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") - testV1Command("CACHE TABLE", t) + testV1CommandSupportingTempView("CACHE TABLE", t) val e = intercept[AnalysisException] { sql(s"CACHE LAZY TABLE $t") } - assert(e.message.contains("CACHE TABLE is only supported with v1 tables")) + assert(e.message.contains("CACHE TABLE is only supported with temp views or v1 tables")) } } @@ -1873,8 +1873,8 @@ class DataSourceV2SQLSuite withTable(t) { sql(s"CREATE TABLE $t (id bigint, data string) USING foo") - testV1Command("UNCACHE TABLE", t) - testV1Command("UNCACHE TABLE", s"IF EXISTS $t") + testV1CommandSupportingTempView("UNCACHE TABLE", t) + testV1CommandSupportingTempView("UNCACHE TABLE", s"IF EXISTS $t") } } @@ -1954,7 +1954,7 @@ class DataSourceV2SQLSuite val e = intercept[AnalysisException] { sql(s"ALTER VIEW $v AS SELECT 1") } - assert(e.message.contains("ALTER VIEW QUERY is only supported with v1 tables")) + assert(e.message.contains("ALTER VIEW QUERY is only supported with temp views or v1 tables")) } test("CREATE VIEW") { @@ -2252,6 +2252,21 @@ class DataSourceV2SQLSuite .head().getString(1) === expectedComment) } + test("SPARK-30799: temp view name can't contain catalog name") { + val sessionCatalogName = CatalogManager.SESSION_CATALOG_NAME + withTempView("v") { + spark.range(10).createTempView("v") + val e1 = intercept[AnalysisException]( + sql(s"CACHE TABLE $sessionCatalogName.v") + ) + assert(e1.message.contains("Table or view not found: default.v")) + } + val e2 = intercept[AnalysisException] { + sql(s"CREATE TEMP VIEW $sessionCatalogName.v AS SELECT 1") + } + assert(e2.message.contains("It is not allowed to add database prefix")) + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") @@ -2259,6 +2274,13 @@ class DataSourceV2SQLSuite assert(e.message.contains(s"$sqlCommand is only supported with v1 tables")) } + private def testV1CommandSupportingTempView(sqlCommand: String, sqlParams: String): Unit = { + val e = intercept[AnalysisException] { + sql(s"$sqlCommand $sqlParams") + } + assert(e.message.contains(s"$sqlCommand is only supported with temp views or v1 tables")) + } + private def assertAnalysisError(sqlStatement: String, expectedError: String): Unit = { val errMsg = intercept[AnalysisException] { sql(sqlStatement) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index d439e5b1cd65..21c698c4a2c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -128,6 +128,7 @@ class PlanResolutionSuite extends AnalysisTest { } }) when(manager.currentCatalog).thenReturn(v2SessionCatalog) + when(manager.currentNamespace).thenReturn(Array.empty[String]) when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) manager } @@ -145,7 +146,7 @@ class PlanResolutionSuite extends AnalysisTest { ResolveInlineTables(conf), analyzer.ResolveRelations, new ResolveCatalogs(catalogManager), - new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), + new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v"), _ => false), analyzer.ResolveTables, analyzer.ResolveReferences, analyzer.ResolveSubqueryColumnAliases, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index b117c582a3e6..8020492ff759 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -75,7 +75,8 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: - new ResolveSessionCatalog(catalogManager, conf, catalog.isView) +: + new ResolveSessionCatalog( + catalogManager, conf, catalog.isTempView, catalog.isTempFunction) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =