Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1115,6 +1115,9 @@ class Analyzer(override val catalogManager: CatalogManager)
executeSameContext(child)
}
}
// Fail the analysis eagerly because outside AnalysisContext, the unresolved operators
// inside a view maybe resolved incorrectly.
checkAnalysis(newChild)
view.copy(child = newChild)
case p @ SubqueryAlias(_, view: View) =>
p.copy(child = resolveViews(view))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.analysis

import java.io.File

import scala.collection.JavaConverters._

import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito._
import org.mockito.invocation.InvocationOnMock
Expand All @@ -27,8 +29,8 @@ import org.scalatest.matchers.must.Matchers
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.connector.InMemoryTableCatalog
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, V1Table}
import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog}
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table}
import org.apache.spark.sql.types._

class TableLookupCacheSuite extends AnalysisTest with Matchers {
Expand All @@ -46,7 +48,12 @@ class TableLookupCacheSuite extends AnalysisTest with Matchers {
ignoreIfExists = false)
val v2Catalog = new InMemoryTableCatalog {
override def loadTable(ident: Identifier): Table = {
V1Table(externalCatalog.getTable("default", ident.name))
val catalogTable = externalCatalog.getTable("default", ident.name)
new InMemoryTable(
catalogTable.identifier.table,
catalogTable.schema,
Array.empty,
Map.empty[String, String].asJava)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

change V1Table to V2Table here, because to lookup V1 table, the catalog will return UnresolvedCatalogRelation first and it needs FindDataSourceTable rule to resolve. But, FindDataSourceTable is in sql/core while the test is in sql/catalyst. So we need to use V2Table instead to avoid depending on FindDataSourceTable

}
override def name: String = CatalogManager.SESSION_CATALOG_NAME
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -258,6 +258,26 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils {
checkViewOutput(viewName, Seq(Row(2)))
}
}

test("SPARK-34490 - query should fail if the view refers a dropped table") {
withTable("t") {
Seq(2, 3, 1).toDF("c1").write.format("parquet").saveAsTable("t")
val viewName = createView("testView", "SELECT * FROM t")
withView(viewName) {
// Always create a temp view in this case, not use `createView` on purpose
sql("CREATE TEMP VIEW t AS SELECT 1 AS c1")
withTempView("t") {
checkViewOutput(viewName, Seq(Row(2), Row(3), Row(1)))
// Manually drop table `t` to see if the query will fail
sql("DROP TABLE IF EXISTS default.t")
val e = intercept[AnalysisException] {
sql(s"SELECT * FROM $viewName").collect()
}.getMessage
assert(e.contains("Table or view not found: t"))
}
}
}
}
}

class LocalTempViewTestSuite extends SQLViewTestSuite with SharedSparkSession {
Expand Down