allisonwang-db commented on a change in pull request #34358:
URL: https://github.com/apache/spark/pull/34358#discussion_r735987452



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1211,82 +1064,124 @@ class Analyzer(override val catalogManager: 
CatalogManager)
           case table => table
         }.getOrElse(u)
 
-      case u @ UnresolvedView(identifier, cmd, _, relationTypeMismatchHint) =>
+      case u @ UnresolvedView(identifier, cmd, allowTemp, 
relationTypeMismatchHint) =>
         lookupTableOrView(identifier).map {
+          case v: ResolvedView if v.isTemp && !allowTemp =>
+            val name = identifier.quoted
+            u.failAnalysis(s"$name is a temp view. '$cmd' expects a permanent 
view.")
           case t: ResolvedTable =>
             throw QueryCompilationErrors.expectViewNotTableError(
               t, cmd, relationTypeMismatchHint, u)
-          case view => view
+          case other => other
         }.getOrElse(u)
 
-      case u @ UnresolvedTableOrView(identifier, _, _) =>
-        lookupTableOrView(identifier).getOrElse(u)
+      case u @ UnresolvedTableOrView(identifier, cmd, allowTempView) =>
+        lookupTableOrView(identifier).map {
+          case v: ResolvedView if v.isTemp && !allowTempView =>
+            throw 
QueryCompilationErrors.expectTableOrPermanentViewNotTempViewError(
+              identifier.quoted, cmd, u)
+          case other => other
+        }.getOrElse(u)
     }
 
-    private def lookupTableOrView(identifier: Seq[String]): 
Option[LogicalPlan] = {
-      expandIdentifier(identifier) match {
-        case SessionCatalogAndIdentifier(catalog, ident) =>
-          CatalogV2Util.loadTable(catalog, ident).map {
-            case v1Table: V1Table if v1Table.v1Table.tableType == 
CatalogTableType.VIEW =>
-              ResolvedView(ident, isTemp = false)
-            case table =>
-              ResolvedTable.create(catalog.asTableCatalog, ident, table)
-          }
+    private def lookupTempView(
+        identifier: Seq[String],
+        isStreaming: Boolean = false): Option[LogicalPlan] = {
+      // We are resolving a view and this name is not a temp view when that 
view was created. We
+      // return None earlier here.
+      if (isResolvingView && !isReferredTempViewName(identifier)) return None
+
+      val tmpView = identifier match {
+        case Seq(part1) => v1SessionCatalog.lookupTempView(part1)
+        case Seq(part1, part2) => v1SessionCatalog.lookupGlobalTempView(part1, 
part2)
         case _ => None
       }
+
+      if (isStreaming && tmpView.nonEmpty && !tmpView.get.isStreaming) {
+        throw 
QueryCompilationErrors.readNonStreamingTempViewError(identifier.quoted)
+      }
+      tmpView
     }
 
-    // Look up a relation from the session catalog with the following logic:
-    // 1) If the resolved catalog is not session catalog, return None.
-    // 2) If a relation is not found in the catalog, return None.
-    // 3) If a v1 table is found, create a v1 relation. Otherwise, create a v2 
relation.
-    private def lookupRelation(
-        identifier: Seq[String],
+    private def lookupTableOrView(identifier: Seq[String]): 
Option[LogicalPlan] = {

Review comment:
       Maybe we can add a comment for this method `lookupTableOrView` saying 
that it is used for DDL commands (return either `ResolvedView` or 
`ResolvedTable`) while `lookupRelation` is used for DML/SELECT queries that 
return a concrete logical plan for the relation (if I understand it correctly).

##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1184,24 +1039,22 @@ class Analyzer(override val catalogManager: 
CatalogManager)
       case write: V2WriteCommand =>
         write.table match {
           case u: UnresolvedRelation if !u.isStreaming =>
-            lookupRelation(u.multipartIdentifier, u.options, false)
-              .map(EliminateSubqueryAliases(_))
-              .map {
-                case v: View => throw 
QueryCompilationErrors.writeIntoViewNotAllowedError(
-                  v.desc.identifier, write)
-                case u: UnresolvedCatalogRelation =>
-                  throw QueryCompilationErrors.writeIntoV1TableNotAllowedError(
-                    u.tableMeta.identifier, write)
-                case r: DataSourceV2Relation => write.withNewTable(r)
-                case other => throw new IllegalStateException(
-                  "[BUG] unexpected plan returned by `lookupRelation`: " + 
other)
-              }.getOrElse(write)
+            lookupRelation(u).map(unwrapRelationPlan).map {
+              case v: View => throw 
QueryCompilationErrors.writeIntoViewNotAllowedError(

Review comment:
       Can this `View` be both temp view and permanent view?

##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -1162,12 +1010,19 @@ class Analyzer(override val catalogManager: 
CatalogManager)
       case _ => plan
     }
 
-    def apply(plan: LogicalPlan): LogicalPlan = ResolveTempViews(plan).
-      resolveOperatorsUpWithPruning(AlwaysProcess.fn, ruleId) {
+    private def unwrapRelationPlan(plan: LogicalPlan): LogicalPlan = {
+      EliminateSubqueryAliases(plan) match {
+        case v: View if v.isTempViewStoringAnalyzedPlan => v.child

Review comment:
       Just curious why this is checking `isTempViewStoringAnalyzedPlan` 
instead of checking `isTempView`? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to