Skip to content

Commit 6ba17cd

Browse files
author
Andrew Or
committed
[SPARK-14414][SQL] Make DDL exceptions more consistent
## What changes were proposed in this pull request? Just a bunch of small tweaks on DDL exception messages. ## How was this patch tested? `DDLCommandSuite` et al. Author: Andrew Or <[email protected]> Closes #12853 from andrewor14/make-exceptions-consistent.
1 parent 9e4928b commit 6ba17cd

File tree

20 files changed

+141
-435
lines changed

20 files changed

+141
-435
lines changed

sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4

Lines changed: 8 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -81,18 +81,8 @@ statement
8181
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions
8282
| ALTER VIEW tableIdentifier
8383
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions
84-
| ALTER TABLE tableIdentifier partitionSpec?
85-
SET FILEFORMAT fileFormat #setTableFileFormat
8684
| ALTER TABLE tableIdentifier partitionSpec? SET locationSpec #setTableLocation
87-
| ALTER TABLE tableIdentifier partitionSpec?
88-
CHANGE COLUMN? oldName=identifier colType
89-
(FIRST | AFTER after=identifier)? (CASCADE | RESTRICT)? #changeColumn
90-
| ALTER TABLE tableIdentifier partitionSpec?
91-
ADD COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)? #addColumns
92-
| ALTER TABLE tableIdentifier partitionSpec?
93-
REPLACE COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)? #replaceColumns
94-
| DROP TABLE (IF EXISTS)? tableIdentifier PURGE?
95-
(FOR METADATA? REPLICATION '(' STRING ')')? #dropTable
85+
| DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable
9686
| DROP VIEW (IF EXISTS)? tableIdentifier #dropTable
9787
| CREATE (OR REPLACE)? VIEW (IF NOT EXISTS)? tableIdentifier
9888
identifierCommentList? (COMMENT STRING)?
@@ -170,6 +160,10 @@ unsupportedHiveNativeCommands
170160
| kw1=ALTER kw2=TABLE tableIdentifier kw3=TOUCH
171161
| kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=COMPACT
172162
| kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CONCATENATE
163+
| kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=SET kw4=FILEFORMAT
164+
| kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=ADD kw4=COLUMNS
165+
| kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CHANGE kw4=COLUMNS?
166+
| kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=REPLACE kw4=COLUMNS
173167
| kw1=START kw2=TRANSACTION
174168
| kw1=COMMIT
175169
| kw1=ROLLBACK
@@ -647,9 +641,9 @@ nonReserved
647641
| NO | DATA
648642
| START | TRANSACTION | COMMIT | ROLLBACK
649643
| SORT | CLUSTER | DISTRIBUTE | UNSET | TBLPROPERTIES | SKEWED | STORED | DIRECTORIES | LOCATION
650-
| EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE | FIRST
651-
| AFTER | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT
652-
| DBPROPERTIES | DFS | TRUNCATE | METADATA | REPLICATION | COMPUTE
644+
| EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE
645+
| CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT
646+
| DBPROPERTIES | DFS | TRUNCATE | COMPUTE
653647
| STATISTICS | ANALYZE | PARTITIONED | EXTERNAL | DEFINED | RECORDWRITER
654648
| REVOKE | GRANT | LOCK | UNLOCK | MSCK | REPAIR | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE
655649
| ROLES | COMPACTIONS | PRINCIPALS | TRANSACTIONS | INDEX | INDEXES | LOCKS | OPTION | LOCAL | INPATH
@@ -836,8 +830,6 @@ TOUCH: 'TOUCH';
836830
COMPACT: 'COMPACT';
837831
CONCATENATE: 'CONCATENATE';
838832
CHANGE: 'CHANGE';
839-
FIRST: 'FIRST';
840-
AFTER: 'AFTER';
841833
CASCADE: 'CASCADE';
842834
RESTRICT: 'RESTRICT';
843835
CLUSTERED: 'CLUSTERED';
@@ -849,8 +841,6 @@ DATABASE: 'DATABASE' | 'SCHEMA';
849841
DATABASES: 'DATABASES' | 'SCHEMAS';
850842
DFS: 'DFS';
851843
TRUNCATE: 'TRUNCATE';
852-
METADATA: 'METADATA';
853-
REPLICATION: 'REPLICATION';
854844
ANALYZE: 'ANALYZE';
855845
COMPUTE: 'COMPUTE';
856846
STATISTICS: 'STATISTICS';

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -25,13 +25,17 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
2525
* Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception
2626
* as an [[org.apache.spark.sql.AnalysisException]] with the correct position information.
2727
*/
28-
class NoSuchDatabaseException(db: String) extends AnalysisException(s"Database $db not found")
28+
class NoSuchDatabaseException(db: String) extends AnalysisException(s"Database '$db' not found")
2929

3030
class NoSuchTableException(db: String, table: String)
31-
extends AnalysisException(s"Table or View $table not found in database $db")
31+
extends AnalysisException(s"Table or view '$table' not found in database '$db'")
3232

33-
class NoSuchPartitionException(db: String, table: String, spec: TablePartitionSpec) extends
34-
AnalysisException(s"Partition not found in table $table database $db:\n" + spec.mkString("\n"))
33+
class NoSuchPartitionException(
34+
db: String,
35+
table: String,
36+
spec: TablePartitionSpec)
37+
extends AnalysisException(
38+
s"Partition not found in table '$table' database '$db':\n" + spec.mkString("\n"))
3539

3640
class NoSuchFunctionException(db: String, func: String)
37-
extends AnalysisException(s"Function $func not found in database $db")
41+
extends AnalysisException(s"Function '$func' not found in database '$db'")

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

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -164,7 +164,7 @@ class InMemoryCatalog extends ExternalCatalog {
164164
catalog(db).tables.remove(table)
165165
} else {
166166
if (!ignoreIfNotExists) {
167-
throw new AnalysisException(s"Table or View '$table' does not exist in database '$db'")
167+
throw new AnalysisException(s"Table or view '$table' does not exist in database '$db'")
168168
}
169169
}
170170
}
@@ -211,7 +211,7 @@ class InMemoryCatalog extends ExternalCatalog {
211211
loadPath: String,
212212
isOverwrite: Boolean,
213213
holdDDLTime: Boolean): Unit = {
214-
throw new AnalysisException("loadTable is not implemented for InMemoryCatalog.")
214+
throw new UnsupportedOperationException("loadTable is not implemented")
215215
}
216216

217217
override def loadPartition(
@@ -223,7 +223,7 @@ class InMemoryCatalog extends ExternalCatalog {
223223
holdDDLTime: Boolean,
224224
inheritTableSpecs: Boolean,
225225
isSkewedStoreAsSubdir: Boolean): Unit = {
226-
throw new AnalysisException("loadPartition is not implemented for InMemoryCatalog.")
226+
throw new UnsupportedOperationException("loadPartition is not implemented.")
227227
}
228228

229229
// --------------------------------------------------------------------------
@@ -304,8 +304,8 @@ class InMemoryCatalog extends ExternalCatalog {
304304
partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = synchronized {
305305
requireTableExists(db, table)
306306
if (partialSpec.nonEmpty) {
307-
throw new AnalysisException("listPartition does not support partition spec in " +
308-
"InMemoryCatalog.")
307+
throw new UnsupportedOperationException(
308+
"listPartition with partial partition spec is not implemented")
309309
}
310310
catalog(db).tables(table).partitions.values.toSeq
311311
}

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

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,6 @@
1717

1818
package org.apache.spark.sql.catalyst.catalog
1919

20-
import java.io.File
21-
2220
import scala.collection.mutable
2321

2422
import org.apache.hadoop.conf.Configuration
@@ -284,10 +282,12 @@ class SessionCatalog(
284282
* This assumes the database specified in `oldName` matches the one specified in `newName`.
285283
*/
286284
def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = {
287-
if (oldName.database != newName.database) {
288-
throw new AnalysisException("rename does not support moving tables across databases")
289-
}
290285
val db = oldName.database.getOrElse(currentDb)
286+
val newDb = newName.database.getOrElse(currentDb)
287+
if (db != newDb) {
288+
throw new AnalysisException(
289+
s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'")
290+
}
291291
val oldTableName = formatTableName(oldName.table)
292292
val newTableName = formatTableName(newName.table)
293293
if (oldName.database.isDefined || !tempTables.contains(oldTableName)) {
@@ -315,7 +315,7 @@ class SessionCatalog(
315315
if (externalCatalog.tableExists(db, table)) {
316316
externalCatalog.dropTable(db, table, ignoreIfNotExists = true)
317317
} else if (!ignoreIfNotExists) {
318-
logError(s"Table or View '${name.quotedString}' does not exist")
318+
throw new AnalysisException(s"Table or view '${name.quotedString}' does not exist")
319319
}
320320
} else {
321321
tempTables.remove(table)
@@ -534,7 +534,7 @@ class SessionCatalog(
534534
if (!functionExists(identifier)) {
535535
externalCatalog.createFunction(db, newFuncDefinition)
536536
} else if (!ignoreIfExists) {
537-
throw new AnalysisException(s"function '$identifier' already exists in database '$db'")
537+
throw new AnalysisException(s"Function '$identifier' already exists in database '$db'")
538538
}
539539
}
540540

@@ -632,9 +632,9 @@ class SessionCatalog(
632632
}
633633

634634
protected def failFunctionLookup(name: String): Nothing = {
635-
throw new AnalysisException(s"Undefined function: $name. This function is " +
635+
throw new AnalysisException(s"Undefined function: '$name'. This function is " +
636636
s"neither a registered temporary function nor " +
637-
s"a permanent function registered in the database $currentDb.")
637+
s"a permanent function registered in the database '$currentDb'.")
638638
}
639639

640640
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,7 @@ object ParserUtils {
3939
stream.getText(Interval.of(0, stream.size()))
4040
}
4141

42-
def parseException(message: String, ctx: ParserRuleContext): ParseException = {
42+
def operationNotAllowed(message: String, ctx: ParserRuleContext): ParseException = {
4343
new ParseException(s"Operation not allowed: $message", ctx)
4444
}
4545

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -233,9 +233,9 @@ class SessionCatalogSuite extends SparkFunSuite {
233233
intercept[AnalysisException] {
234234
catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = true)
235235
}
236-
// If the table does not exist, we do not issue an exception. Instead, we output an error log
237-
// message to console when ignoreIfNotExists is set to false.
238-
catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false)
236+
intercept[AnalysisException] {
237+
catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false)
238+
}
239239
catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = true)
240240
}
241241

0 commit comments

Comments
 (0)