Skip to content

Commit 9bd844b

Browse files
ganeshas-dbcloud-fan
authored andcommitted
[SPARK-53558][SQL] Show fully qualified table name including the catalog name in the exception message when the table is not found
Currently, we only display the database name and table name in the error message when the table is not found. We should display the fully qualified table name, including the catalog name, in the exception message when the table is missing. ### What changes were proposed in this pull request? ### Why are the changes needed? Better error message. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Modified the affected unit tests and verified that the error message has the fully qualified table name when the table is not found. ### Was this patch authored or co-authored using generative AI tooling? No Closes #52315 from ganeshashree/SPARK-53558. Authored-by: Ganesha S <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 19ca63f commit 9bd844b

File tree

9 files changed

+23
-17
lines changed

9 files changed

+23
-17
lines changed

sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -182,7 +182,7 @@ default Table loadTable(
182182
* @throws NoSuchTableException If the table doesn't exist or is a view
183183
*/
184184
default Table loadTable(Identifier ident, String version) throws NoSuchTableException {
185-
throw QueryCompilationErrors.noSuchTableError(ident);
185+
throw QueryCompilationErrors.noSuchTableError(name(), ident);
186186
}
187187

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

203203
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -268,7 +268,8 @@ class SessionCatalog(
268268

269269
private def requireTableExists(name: TableIdentifier): Unit = {
270270
if (!tableExists(name)) {
271-
throw new NoSuchTableException(db = name.database.get, table = name.table)
271+
throw new NoSuchTableException(
272+
Seq(name.catalog.get, name.database.get, name.table))
272273
}
273274
}
274275

sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1511,8 +1511,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
15111511
"continuousSources" -> continuousSources.mkString(", ")))
15121512
}
15131513

1514-
def noSuchTableError(ident: Identifier): NoSuchTableException = {
1515-
new NoSuchTableException(ident.asMultipartIdentifier)
1514+
def noSuchTableError(catalogName: String, ident: Identifier): NoSuchTableException = {
1515+
new NoSuchTableException(catalogName +: ident.asMultipartIdentifier)
15161516
}
15171517

15181518
def noSuchTableError(nameParts: Seq[String]): Throwable = {

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -850,14 +850,16 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
850850
catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1"))
851851
},
852852
condition = "TABLE_OR_VIEW_NOT_FOUND",
853-
parameters = Map("relationName" -> "`default`.`view1`")
853+
parameters = Map("relationName" ->
854+
s"`${CatalogManager.SESSION_CATALOG_NAME}`.`default`.`view1`")
854855
)
855856
checkError(
856857
exception = intercept[NoSuchTableException] {
857858
catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default")))
858859
},
859860
condition = "TABLE_OR_VIEW_NOT_FOUND",
860-
parameters = Map("relationName" -> "`default`.`view1`")
861+
parameters = Map("relationName" ->
862+
s"`${CatalogManager.SESSION_CATALOG_NAME}`.`default`.`view1`")
861863
)
862864

863865
createTempView(catalog, "view1", tempTable, overrideIfExists = false)
@@ -871,7 +873,8 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually {
871873
catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default")))
872874
},
873875
condition = "TABLE_OR_VIEW_NOT_FOUND",
874-
parameters = Map("relationName" -> "`default`.`view1`")
876+
parameters = Map("relationName" ->
877+
s"`${CatalogManager.SESSION_CATALOG_NAME}`.`default`.`view1`")
875878
)
876879
}
877880
}

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -126,7 +126,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
126126
}
127127
} catch {
128128
case _: NoSuchNamespaceException =>
129-
throw QueryCompilationErrors.noSuchTableError(ident)
129+
throw QueryCompilationErrors.noSuchTableError(name(), ident)
130130
}
131131
}
132132

@@ -284,7 +284,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
284284
catalog.getTableMetadata(ident.asTableIdentifier)
285285
} catch {
286286
case _: NoSuchTableException =>
287-
throw QueryCompilationErrors.noSuchTableError(ident)
287+
throw QueryCompilationErrors.noSuchTableError(name(), ident)
288288
}
289289

290290
val properties = CatalogV2Util.applyPropertiesChanges(catalogTable.properties, changes)
@@ -313,7 +313,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
313313
}
314314
} catch {
315315
case _: NoSuchTableException =>
316-
throw QueryCompilationErrors.noSuchTableError(ident)
316+
throw QueryCompilationErrors.noSuchTableError(name(), ident)
317317
}
318318

319319
null // Return null to save the `loadTable` call for ALTER TABLE.

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -142,7 +142,7 @@ class JDBCTableCatalog extends TableCatalog
142142
override def loadTable(ident: Identifier): Table = {
143143
JdbcUtils.withConnection(options) { conn =>
144144
if (!tableExists(ident, conn)) {
145-
throw QueryCompilationErrors.noSuchTableError(ident)
145+
throw QueryCompilationErrors.noSuchTableError(name(), ident)
146146
}
147147

148148
val optionsWithTableName = new JDBCOptions(

sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -139,7 +139,7 @@ class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable
139139

140140
newTable
141141
case _ =>
142-
throw QueryCompilationErrors.noSuchTableError(ident)
142+
throw QueryCompilationErrors.noSuchTableError(name(), ident)
143143
}
144144
}
145145
}

sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -221,7 +221,8 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with
221221
val e = intercept[NoSuchTableException] {
222222
spark.read.format(format).option("name", "non_existent_table").load()
223223
}
224-
checkErrorTableNotFound(e, "`default`.`non_existent_table`")
224+
val currentCatalogName = spark.catalog.currentCatalog().name()
225+
checkErrorTableNotFound(e, s"`$currentCatalogName`.`default`.`non_existent_table`")
225226
}
226227

227228
test("DataFrameWriter creates v2Relation with identifiers") {

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ import org.apache.spark.sql.AnalysisException
3131
import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException}
3232
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
3333
import org.apache.spark.sql.catalyst.util.quoteIdentifier
34-
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, TableSummary, V1Table}
34+
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, Column, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, TableSummary, V1Table}
3535
import org.apache.spark.sql.connector.expressions.Transform
3636
import org.apache.spark.sql.test.SharedSparkSession
3737
import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType, TimestampType}
@@ -48,8 +48,9 @@ abstract class V2SessionCatalogBaseSuite extends SharedSparkSession with BeforeA
4848
val testNs: Array[String] = Array("db")
4949
val defaultNs: Array[String] = Array("default")
5050
val testIdent: Identifier = Identifier.of(testNs, "test_table")
51-
val testIdentQuoted: String = (testIdent.namespace :+ testIdent.name)
52-
.map(part => quoteIdentifier(part)).mkString(".")
51+
val catalogName: String = CatalogManager.SESSION_CATALOG_NAME
52+
val testIdentQuoted: String = (catalogName +: testIdent.namespace :+ testIdent.name)
53+
.map(quoteIdentifier).mkString(".")
5354

5455
def newCatalog(): V2SessionCatalog = {
5556
val newCatalog = new V2SessionCatalog(spark.sessionState.catalog)

0 commit comments

Comments
 (0)