Skip to content

Commit 6d8c438

Browse files
committed
[SPARK-53558][SQL] Show fully qualified table name including the catalog name in the exception message when the table is not found
1 parent bb41e19 commit 6d8c438

File tree

8 files changed

+17
-14
lines changed

8 files changed

+17
-14
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/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)