Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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 @@ -182,7 +182,7 @@ default Table loadTable(
* @throws NoSuchTableException If the table doesn't exist or is a view
*/
default Table loadTable(Identifier ident, String version) throws NoSuchTableException {
throw QueryCompilationErrors.noSuchTableError(ident);
throw QueryCompilationErrors.noSuchTableError(name(), ident);
}

/**
Expand All @@ -197,7 +197,7 @@ default Table loadTable(Identifier ident, String version) throws NoSuchTableExce
* @throws NoSuchTableException If the table doesn't exist or is a view
*/
default Table loadTable(Identifier ident, long timestamp) throws NoSuchTableException {
throw QueryCompilationErrors.noSuchTableError(ident);
throw QueryCompilationErrors.noSuchTableError(name(), ident);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -268,7 +268,8 @@ class SessionCatalog(

private def requireTableExists(name: TableIdentifier): Unit = {
if (!tableExists(name)) {
throw new NoSuchTableException(db = name.database.get, table = name.table)
throw new NoSuchTableException(
Seq(name.catalog.get, name.database.get, name.table))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1511,8 +1511,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
"continuousSources" -> continuousSources.mkString(", ")))
}

def noSuchTableError(ident: Identifier): NoSuchTableException = {
new NoSuchTableException(ident.asMultipartIdentifier)
def noSuchTableError(catalogName: String, ident: Identifier): NoSuchTableException = {
new NoSuchTableException(catalogName +: ident.asMultipartIdentifier)
}

def noSuchTableError(nameParts: Seq[String]): Throwable = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -850,14 +850,16 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1"))
},
condition = "TABLE_OR_VIEW_NOT_FOUND",
parameters = Map("relationName" -> "`default`.`view1`")
parameters = Map("relationName" ->
s"`${CatalogManager.SESSION_CATALOG_NAME}`.`default`.`view1`")
)
checkError(
exception = intercept[NoSuchTableException] {
catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default")))
},
condition = "TABLE_OR_VIEW_NOT_FOUND",
parameters = Map("relationName" -> "`default`.`view1`")
parameters = Map("relationName" ->
s"`${CatalogManager.SESSION_CATALOG_NAME}`.`default`.`view1`")
)

createTempView(catalog, "view1", tempTable, overrideIfExists = false)
Expand All @@ -871,7 +873,8 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default")))
},
condition = "TABLE_OR_VIEW_NOT_FOUND",
parameters = Map("relationName" -> "`default`.`view1`")
parameters = Map("relationName" ->
s"`${CatalogManager.SESSION_CATALOG_NAME}`.`default`.`view1`")
)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
}
} catch {
case _: NoSuchNamespaceException =>
throw QueryCompilationErrors.noSuchTableError(ident)
throw QueryCompilationErrors.noSuchTableError(name(), ident)
}
}

Expand Down Expand Up @@ -284,7 +284,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
catalog.getTableMetadata(ident.asTableIdentifier)
} catch {
case _: NoSuchTableException =>
throw QueryCompilationErrors.noSuchTableError(ident)
throw QueryCompilationErrors.noSuchTableError(name(), ident)
}

val properties = CatalogV2Util.applyPropertiesChanges(catalogTable.properties, changes)
Expand Down Expand Up @@ -313,7 +313,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
}
} catch {
case _: NoSuchTableException =>
throw QueryCompilationErrors.noSuchTableError(ident)
throw QueryCompilationErrors.noSuchTableError(name(), ident)
}

null // Return null to save the `loadTable` call for ALTER TABLE.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ class JDBCTableCatalog extends TableCatalog
override def loadTable(ident: Identifier): Table = {
JdbcUtils.withConnection(options) { conn =>
if (!tableExists(ident, conn)) {
throw QueryCompilationErrors.noSuchTableError(ident)
throw QueryCompilationErrors.noSuchTableError(name(), ident)
}

val optionsWithTableName = new JDBCOptions(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,7 @@ class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable

newTable
case _ =>
throw QueryCompilationErrors.noSuchTableError(ident)
throw QueryCompilationErrors.noSuchTableError(name(), ident)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,8 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with
val e = intercept[NoSuchTableException] {
spark.read.format(format).option("name", "non_existent_table").load()
}
checkErrorTableNotFound(e, "`default`.`non_existent_table`")
val currentCatalogName = spark.catalog.currentCatalog().name()
checkErrorTableNotFound(e, s"`$currentCatalogName`.`default`.`non_existent_table`")
}

test("DataFrameWriter creates v2Relation with identifiers") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException}
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.util.quoteIdentifier
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, TableSummary, V1Table}
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, Column, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, TableSummary, V1Table}
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType, TimestampType}
Expand All @@ -48,8 +48,9 @@ abstract class V2SessionCatalogBaseSuite extends SharedSparkSession with BeforeA
val testNs: Array[String] = Array("db")
val defaultNs: Array[String] = Array("default")
val testIdent: Identifier = Identifier.of(testNs, "test_table")
val testIdentQuoted: String = (testIdent.namespace :+ testIdent.name)
.map(part => quoteIdentifier(part)).mkString(".")
val catalogName: String = CatalogManager.SESSION_CATALOG_NAME
val testIdentQuoted: String = (catalogName +: testIdent.namespace :+ testIdent.name)
.map(quoteIdentifier).mkString(".")

def newCatalog(): V2SessionCatalog = {
val newCatalog = new V2SessionCatalog(spark.sessionState.catalog)
Expand Down