From 5c8b8922d8b96cc406f9b462c7368095c2167e2a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 4 Sep 2016 16:29:09 +0800 Subject: [PATCH] should not allow specify database in table/view name after RENAME TO --- .../sql/catalyst/catalog/SessionCatalog.scala | 16 ++------- .../catalog/SessionCatalogSuite.scala | 27 +++++--------- .../spark/sql/execution/SparkSqlParser.scala | 10 ++++-- .../spark/sql/execution/command/tables.scala | 7 ++-- .../execution/command/DDLCommandSuite.scala | 9 +++-- .../sql/execution/command/DDLSuite.scala | 35 ++----------------- 6 files changed, 32 insertions(+), 72 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e7132cd3975d5..9fb5db573b70f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -349,29 +349,17 @@ class SessionCatalog( * If a database is specified in `oldName`, this will rename the table in that database. * If no database is specified, this will first attempt to rename a temporary table with * the same name, then, if that does not exist, rename the table in the current database. - * - * This assumes the database specified in `oldName` matches the one specified in `newName`. */ - def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = synchronized { + def renameTable(oldName: TableIdentifier, newName: String): Unit = synchronized { val db = formatDatabaseName(oldName.database.getOrElse(currentDb)) requireDbExists(db) - val newDb = formatDatabaseName(newName.database.getOrElse(currentDb)) - if (db != newDb) { - throw new AnalysisException( - s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'") - } val oldTableName = formatTableName(oldName.table) - val newTableName = formatTableName(newName.table) + val newTableName = formatTableName(newName) if (oldName.database.isDefined || !tempTables.contains(oldTableName)) { requireTableExists(TableIdentifier(oldTableName, Some(db))) requireTableNotExists(TableIdentifier(newTableName, Some(db))) externalCatalog.renameTable(db, oldTableName, newTableName) } else { - if (newName.database.isDefined) { - throw new AnalysisException( - s"RENAME TEMPORARY TABLE from '$oldName' to '$newName': cannot specify database " + - s"name '${newName.database.get}' in the destination table") - } if (tempTables.contains(newTableName)) { throw new AnalysisException( s"RENAME TEMPORARY TABLE from '$oldName' to '$newName': destination table already exists") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index c9d4fef8056ca..012df629bbdef 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -273,37 +273,27 @@ class SessionCatalogSuite extends SparkFunSuite { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) - sessionCatalog.renameTable( - TableIdentifier("tbl1", Some("db2")), TableIdentifier("tblone", Some("db2"))) + sessionCatalog.renameTable(TableIdentifier("tbl1", Some("db2")), "tblone") assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbl2")) - sessionCatalog.renameTable( - TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbltwo", Some("db2"))) + sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), "tbltwo") assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbltwo")) // Rename table without explicitly specifying database sessionCatalog.setCurrentDatabase("db2") - sessionCatalog.renameTable(TableIdentifier("tbltwo"), TableIdentifier("table_two")) + sessionCatalog.renameTable(TableIdentifier("tbltwo"), "table_two") assert(externalCatalog.listTables("db2").toSet == Set("tblone", "table_two")) - // Renaming "db2.tblone" to "db1.tblones" should fail because databases don't match - intercept[AnalysisException] { - sessionCatalog.renameTable( - TableIdentifier("tblone", Some("db2")), TableIdentifier("tblones", Some("db1"))) - } // The new table already exists intercept[TableAlreadyExistsException] { - sessionCatalog.renameTable( - TableIdentifier("tblone", Some("db2")), TableIdentifier("table_two", Some("db2"))) + sessionCatalog.renameTable(TableIdentifier("tblone", Some("db2")), "table_two") } } test("rename table when database/table does not exist") { val catalog = new SessionCatalog(newBasicCatalog()) intercept[NoSuchDatabaseException] { - catalog.renameTable( - TableIdentifier("tbl1", Some("unknown_db")), TableIdentifier("tbl2", Some("unknown_db"))) + catalog.renameTable(TableIdentifier("tbl1", Some("unknown_db")), "tbl2") } intercept[NoSuchTableException] { - catalog.renameTable( - TableIdentifier("unknown_table", Some("db2")), TableIdentifier("tbl2", Some("db2"))) + catalog.renameTable(TableIdentifier("unknown_table", Some("db2")), "tbl2") } } @@ -316,13 +306,12 @@ class SessionCatalogSuite extends SparkFunSuite { assert(sessionCatalog.getTempTable("tbl1") == Option(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be renamed first - sessionCatalog.renameTable(TableIdentifier("tbl1"), TableIdentifier("tbl3")) + sessionCatalog.renameTable(TableIdentifier("tbl1"), "tbl3") assert(sessionCatalog.getTempTable("tbl1").isEmpty) assert(sessionCatalog.getTempTable("tbl3") == Option(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is specified, temp tables are never renamed - sessionCatalog.renameTable( - TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4", Some("db2"))) + sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), "tbl4") assert(sessionCatalog.getTempTable("tbl3") == Option(tempTable)) assert(sessionCatalog.getTempTable("tbl4").isEmpty) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8fc1a8595a45f..fc078da07dcc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -666,9 +666,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { + val fromName = visitTableIdentifier(ctx.from) + val toName = visitTableIdentifier(ctx.to) + if (toName.database.isDefined) { + operationNotAllowed("Can not specify database in table/view name after RENAME TO", ctx) + } + AlterTableRenameCommand( - visitTableIdentifier(ctx.from), - visitTableIdentifier(ctx.to), + fromName, + toName.table, ctx.VIEW != null) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index a1371582be064..4e6caae85caeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -152,7 +152,7 @@ case class CreateTableCommand(table: CatalogTable, ifNotExists: Boolean) extends */ case class AlterTableRenameCommand( oldName: TableIdentifier, - newName: TableIdentifier, + newName: String, isView: Boolean) extends RunnableCommand { @@ -165,6 +165,7 @@ case class AlterTableRenameCommand( if (isTemporary) { catalog.renameTable(oldName, newName) } else { + val newTblName = TableIdentifier(newName, oldName.database) // If an exception is thrown here we can just assume the table is uncached; // this can happen with Hive tables when the underlying catalog is in-memory. val wasCached = Try(sparkSession.catalog.isCached(oldName.unquotedString)).getOrElse(false) @@ -178,7 +179,7 @@ case class AlterTableRenameCommand( // For datasource tables, we also need to update the "path" serde property val table = catalog.getTableMetadata(oldName) if (DDLUtils.isDatasourceTable(table) && table.tableType == CatalogTableType.MANAGED) { - val newPath = catalog.defaultTablePath(newName) + val newPath = catalog.defaultTablePath(newTblName) val newTable = table.withNewStorage( serdeProperties = table.storage.properties ++ Map("path" -> newPath)) catalog.alterTable(newTable) @@ -188,7 +189,7 @@ case class AlterTableRenameCommand( catalog.refreshTable(oldName) catalog.renameTable(oldName, newName) if (wasCached) { - sparkSession.catalog.cacheTable(newName.unquotedString) + sparkSession.catalog.cacheTable(newTblName.unquotedString) } } Seq.empty[Row] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 8dd883b37bde0..547fb63813750 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -388,14 +388,19 @@ class DDLCommandSuite extends PlanTest { val parsed_view = parser.parsePlan(sql_view) val expected_table = AlterTableRenameCommand( TableIdentifier("table_name", None), - TableIdentifier("new_table_name", None), + "new_table_name", isView = false) val expected_view = AlterTableRenameCommand( TableIdentifier("table_name", None), - TableIdentifier("new_table_name", None), + "new_table_name", isView = true) comparePlans(parsed_table, expected_table) comparePlans(parsed_view, expected_view) + + val e = intercept[ParseException]( + parser.parsePlan("ALTER TABLE db1.tbl RENAME TO db1.tbl2") + ) + assert(e.getMessage.contains("Can not specify database in table/view name after RENAME TO")) } // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 0073659a31541..fd35c987cab59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -657,7 +657,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { createDatabase(catalog, "dby") createTable(catalog, tableIdent1) assert(catalog.listTables("dbx") == Seq(tableIdent1)) - sql("ALTER TABLE dbx.tab1 RENAME TO dbx.tab2") + sql("ALTER TABLE dbx.tab1 RENAME TO tab2") assert(catalog.listTables("dbx") == Seq(tableIdent2)) catalog.setCurrentDatabase("dbx") // rename without explicitly specifying database @@ -665,11 +665,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(catalog.listTables("dbx") == Seq(tableIdent1)) // table to rename does not exist intercept[AnalysisException] { - sql("ALTER TABLE dbx.does_not_exist RENAME TO dbx.tab2") - } - // destination database is different - intercept[AnalysisException] { - sql("ALTER TABLE dbx.tab1 RENAME TO dby.tab2") + sql("ALTER TABLE dbx.does_not_exist RENAME TO tab2") } } @@ -691,31 +687,6 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(spark.table("teachers").collect().toSeq == df.collect().toSeq) } - test("rename temporary table - destination table with database name") { - withTempView("tab1") { - sql( - """ - |CREATE TEMPORARY TABLE tab1 - |USING org.apache.spark.sql.sources.DDLScanSource - |OPTIONS ( - | From '1', - | To '10', - | Table 'test1' - |) - """.stripMargin) - - val e = intercept[AnalysisException] { - sql("ALTER TABLE tab1 RENAME TO default.tab2") - } - assert(e.getMessage.contains( - "RENAME TEMPORARY TABLE from '`tab1`' to '`default`.`tab2`': " + - "cannot specify database name 'default' in the destination table")) - - val catalog = spark.sessionState.catalog - assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"))) - } - } - test("rename temporary table - destination table already exists") { withTempView("tab1", "tab2") { sql( @@ -744,7 +715,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("ALTER TABLE tab1 RENAME TO tab2") } assert(e.getMessage.contains( - "RENAME TEMPORARY TABLE from '`tab1`' to '`tab2`': destination table already exists")) + "RENAME TEMPORARY TABLE from '`tab1`' to 'tab2': destination table already exists")) val catalog = spark.sessionState.catalog assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"), TableIdentifier("tab2")))