From 52ca9020c35ed52b59b8e40d7b9cfd8c78a72894 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Mon, 21 Nov 2016 10:41:15 -0800 Subject: [PATCH 01/34] alter_add_col: initial changes --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 +- .../spark/sql/execution/SparkSqlParser.scala | 15 ++++++++ .../spark/sql/execution/command/tables.scala | 37 +++++++++++++++++++ .../spark/sql/hive/HiveExternalCatalog.scala | 20 +++++++--- 4 files changed, 69 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index cc3b8fd3b468..c4a590ec6916 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -85,6 +85,8 @@ statement LIKE source=tableIdentifier locationSpec? #createTableLike | ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS (identifier | FOR COLUMNS identifierSeq)? #analyze + | ALTER TABLE tableIdentifier + ADD COLUMNS '(' columns=colTypeList ')' #addTableColumns | ALTER (TABLE | VIEW) from=tableIdentifier RENAME TO to=tableIdentifier #renameTable | ALTER (TABLE | VIEW) tableIdentifier @@ -198,7 +200,6 @@ unsupportedHiveNativeCommands | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=COMPACT | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CONCATENATE | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=SET kw4=FILEFORMAT - | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=ADD kw4=COLUMNS | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=REPLACE kw4=COLUMNS | kw1=START kw2=TRANSACTION | kw1=COMMIT 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 abea7a3bcf14..52c85a555702 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 @@ -741,6 +741,21 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.VIEW != null) } + /** + * Create a [[AlterTableAddColumnsCommand]] command. + * + * For example: + * {{{ + * ALTER TABLE table1 ADD COLUMNS (c1 int); + * }}} + */ + override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) { + AlterTableAddColumnsCommand( + visitTableIdentifier(ctx.tableIdentifier), + Option(ctx.columns).map(visitColTypeList).getOrElse(Nil) + ) + } + /** * Create an [[AlterTableSetPropertiesCommand]] command. * 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 beb3dcafd64f..619d6d6978e9 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 @@ -174,6 +174,43 @@ case class AlterTableRenameCommand( } +case class AlterTableAddColumnsCommand( + table: TableIdentifier, + columns: Seq[StructField]) extends RunnableCommand { + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + + if (catalog.isTemporaryTable(table)) { + throw new AnalysisException( + s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.") + } else { + val catalogTable = catalog.getTableMetadata(table) + if (catalogTable.tableType == VIEW) { + throw new AnalysisException( + s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.") + } + // 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(table.unquotedString)).getOrElse(false) + if (wasCached) { + try { + sparkSession.catalog.uncacheTable(table.unquotedString) + } catch { + case NonFatal(e) => log.warn(e.toString, e) + } + } + // Invalidate the table last, otherwise uncaching the table would load the logical plan + // back into the hive metastore cache + catalog.refreshTable(table) + + val newSchema = catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns) + catalog.alterTable(catalogTable.copy(schema = newSchema)) + } + Seq.empty[Row] + } +} + + /** * A command that loads data into a Hive table. * diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 33b21be37203..570f68d4592c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -411,7 +411,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } } - properties + properties.toMap } private def defaultTablePath(tableIdent: TableIdentifier): String = { @@ -518,7 +518,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat verifyTableProperties(tableDefinition) // convert table statistics to properties so that we can persist them through hive api - val withStatsProps = if (tableDefinition.stats.isDefined) { + val withStatsProps: CatalogTable = if (tableDefinition.stats.isDefined) { val stats = tableDefinition.stats.get var statsProperties: Map[String, String] = Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) @@ -540,6 +540,12 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } else { val oldTableDef = getRawTable(db, withStatsProps.identifier.table) + val (tableSchema, schemaChange) = if (oldTableDef.schema.equals(withStatsProps.schema)) { + (oldTableDef.schema, false) + } else { + (withStatsProps.schema, true) + } + val newStorage = if (DDLUtils.isHiveTable(tableDefinition)) { tableDefinition.storage } else { @@ -591,11 +597,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition, // to retain the spark specific format if it is. Also add old data source properties to table // properties, to retain the data source table format. - val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) - val newTableProps = oldDataSourceProps ++ withStatsProps.properties + partitionProviderProp + val dataSourceProps = if (schemaChange) { + tableMetaToTableProps(withStatsProps).filter(_._1.startsWith(DATASOURCE_PREFIX)) + } else { + oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) + } + val newTableProps = dataSourceProps ++ withStatsProps.properties + partitionProviderProp val newDef = withStatsProps.copy( storage = newStorage, - schema = oldTableDef.schema, + schema = tableSchema, partitionColumnNames = oldTableDef.partitionColumnNames, bucketSpec = oldTableDef.bucketSpec, properties = newTableProps) From f498fa696ba21d93e224b2ed93d2308d9a1392cd Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Thu, 1 Dec 2016 14:03:28 -0800 Subject: [PATCH 02/34] add testcases --- .../SpecificParquetRecordReaderBase.java | 8 +- .../spark/sql/execution/command/ddl.scala | 24 ++++++ .../spark/sql/execution/command/tables.scala | 42 ++++------ .../sql/hive/execution/HiveDDLSuite.scala | 83 ++++++++++++++++++- 4 files changed, 130 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index bf8717483575..88da06634d3d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -107,7 +107,13 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont footer = readFooter(configuration, file, range(split.getStart(), split.getEnd())); MessageType fileSchema = footer.getFileMetaData().getSchema(); FilterCompat.Filter filter = getFilter(configuration); - blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema); + try { + blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema); + } catch (IllegalArgumentException e) { + // In the case where a particular parquet files does not contain + // the column(s) in the filter, we don't do filtering at this level + blocks = footer.getBlocks(); + } } else { // otherwise we find the row groups that were selected on the client footer = readFooter(configuration, file, NO_FILTER); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 9d3c55060dfb..305a7cd66dcb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -815,4 +815,28 @@ object DDLUtils { } } } + + /** + * ALTER TABLE ADD COLUMNS command does not support temporary view/table, + * view, or datasource table yet. + */ + def verifyAlterTableAddColumn( + catalog: SessionCatalog, + table: TableIdentifier): CatalogTable = { + if (catalog.isTemporaryTable(table)) { + throw new AnalysisException( + s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.") + } + + val catalogTable = catalog.getTableMetadata(table) + if (catalogTable.tableType == CatalogTableType.VIEW) { + throw new AnalysisException( + s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.") + } + if (isDatasourceTable(catalogTable)) { + throw new AnalysisException( + s"${table.toString} is a datasource table, which does not support ALTER ADD COLUMNS yet.") + } + catalogTable + } } 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 619d6d6978e9..a9249355c4f3 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 @@ -179,33 +179,25 @@ case class AlterTableAddColumnsCommand( columns: Seq[StructField]) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - - if (catalog.isTemporaryTable(table)) { - throw new AnalysisException( - s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.") - } else { - val catalogTable = catalog.getTableMetadata(table) - if (catalogTable.tableType == VIEW) { - throw new AnalysisException( - s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.") - } - // 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(table.unquotedString)).getOrElse(false) - if (wasCached) { - try { - sparkSession.catalog.uncacheTable(table.unquotedString) - } catch { - case NonFatal(e) => log.warn(e.toString, e) - } + val catalogTable = DDLUtils.verifyAlterTableAddColumn(catalog, table) + + // 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(table.unquotedString)).getOrElse(false) + if (wasCached) { + try { + sparkSession.catalog.uncacheTable(table.unquotedString) + } catch { + case NonFatal(e) => log.warn(e.toString, e) } - // Invalidate the table last, otherwise uncaching the table would load the logical plan - // back into the hive metastore cache - catalog.refreshTable(table) - - val newSchema = catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns) - catalog.alterTable(catalogTable.copy(schema = newSchema)) } + // Invalidate the table last, otherwise uncaching the table would load the logical plan + // back into the hive metastore cache + catalog.refreshTable(table) + + val newSchema = catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns) + catalog.alterTable(catalogTable.copy(schema = newSchema)) + Seq.empty[Row] } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index d752c415c1ed..72ce4162b590 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.types.{MetadataBuilder, StructType} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} // TODO(gatorsmile): combine HiveCatalogedDDLSuite and HiveDDLSuite class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeAndAfterEach { @@ -1860,4 +1860,85 @@ class HiveDDLSuite } } } + + Seq("a b", "a:b", "a%b").foreach { specialChars => + test(s"location uri contains $specialChars for database") { + try { + withTable("t") { + withTempDir { dir => + val loc = new File(dir, specialChars) + spark.sql(s"CREATE DATABASE tmpdb LOCATION '$loc'") + spark.sql("USE tmpdb") + + Seq(1).toDF("a").write.saveAsTable("t") + val tblloc = new File(loc, "t") + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + val tblPath = new Path(tblloc.getAbsolutePath) + val fs = tblPath.getFileSystem(spark.sessionState.newHadoopConf()) + assert(table.location == fs.makeQualified(tblPath).toUri) + assert(tblloc.listFiles().nonEmpty) + } + } + } finally { + spark.sql("DROP DATABASE IF EXISTS tmpdb") + } + } + } + + test("alter table add columns -- partitioned") { + val tableTypes = Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO") + tableTypes.foreach{ tableType => + withTable("alter_add_partitioned") { + sql( + s""" + |CREATE TABLE alter_add_partitioned (c1 int, c2 int) + |PARTITIONED BY (c3 int) STORED AS $tableType + """.stripMargin) + + sql("INSERT INTO alter_add_partitioned PARTITION (c3=1) VALUES (1, 2)") + sql("ALTER TABLE alter_add_partitioned ADD COLUMNS (c4 int)") + checkAnswer( + sql("SELECT * FROM alter_add_partitioned WHERE c3 = 1"), + Seq(Row(1, 2, null, 1)) + ) + assert(sql("SELECT * FROM alter_add_partitioned").schema + .contains(StructField("c4", IntegerType))) + sql("INSERT INTO alter_add_partitioned PARTITION (c3=2) VALUES (2, 3, 4)") + checkAnswer( + sql("SELECT * FROM alter_add_partitioned"), + Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2)) + ) + checkAnswer( + sql("SELECT * FROM alter_add_partitioned WHERE c3 = 2 AND c4 IS NOT NULL"), + Seq(Row(2, 3, 4, 2)) + ) + } + } + } + + test("alter table add columns -- with predicate") { + val tableTypes = Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO") + tableTypes.foreach { tableType => + withTable("alter_add_predicate") { + sql(s"CREATE TABLE alter_add_predicate (c1 int, c2 int) STORED AS $tableType") + sql("INSERT INTO alter_add_predicate VALUES (1, 2)") + sql("ALTER TABLE alter_add_predicate ADD COLUMNS (c4 int)") + checkAnswer( + sql("SELECT * FROM alter_add_predicate WHERE c4 IS NULL"), + Seq(Row(1, 2, null)) + ) + assert(sql("SELECT * FROM alter_add_predicate").schema + .contains(StructField("c4", IntegerType))) + sql("INSERT INTO alter_add_predicate VALUES (2, 3, 4)") + checkAnswer( + sql("SELECT * FROM alter_add_predicate WHERE c4 = 4 "), + Seq(Row(2, 3, 4)) + ) + checkAnswer( + sql("SELECT * FROM alter_add_predicate"), + Seq(Row(1, 2, null), Row(2, 3, 4)) + ) + } + } + } } From 522443eef75b5b978a607e7f5820e74142b2cd3c Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Thu, 1 Dec 2016 15:00:38 -0800 Subject: [PATCH 03/34] negative testcases --- .../sql/hive/execution/HiveDDLSuite.scala | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 72ce4162b590..04b12407b9c4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1941,4 +1941,36 @@ class HiveDDLSuite } } } + + test("alter table add columns -- not support temp view") { + withTempView("tmp_v") { + sql("create temporary view tmp_v as select 1 as c1, 2 as c2") + val e = intercept[AnalysisException] { + sql("alter table tmp_v add columns (c3 int)") + } + assert(e.message.contains("is a temporary VIEW, which does not support ALTER ADD COLUMNS")) + } + } + + test("alter table add columns -- not support view") { + withView("v1") { + sql("create view v1 as select 1 as c1, 2 as c2") + val e = intercept[AnalysisException] { + sql("alter table v1 add columns (c3 int)") + } + assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) + } + } + + test("alter table add columns -- not datasource table") { + withTempDir { dir => + withTable("t_ds") { + sql(s"create table t_ds (c1 int, c2 int) using parquet options(path '$dir')") + val e = intercept[AnalysisException] { + sql("alter table t_ds add columns (c3 int)") + } + assert(e.message.contains("datasource table, which does not support ALTER ADD COLUMNS yet")) + } + } + } } From 1af2654c43c46fdd7cc3dcf1d57b4fd2b8b81082 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Mon, 5 Dec 2016 10:35:14 -0800 Subject: [PATCH 04/34] remove non-support testcase --- .../sql/execution/command/DDLCommandSuite.scala | 13 +++++++++++++ .../spark/sql/hive/HiveExternalCatalog.scala | 14 +++++++++----- 2 files changed, 22 insertions(+), 5 deletions(-) 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 4b73b078da38..725b88c9ba31 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 @@ -794,6 +794,19 @@ class DDLCommandSuite extends PlanTest { """.stripMargin) } + test("alter table: change column name/type/position/comment (not allowed)") { + assertUnsupported("ALTER TABLE table_name CHANGE col_old_name col_new_name INT") + assertUnsupported( + """ + |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT + |COMMENT 'col_comment' FIRST CASCADE + """.stripMargin) + assertUnsupported(""" + |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT + |COMMENT 'col_comment' AFTER column_name RESTRICT + """.stripMargin) + } + test("show databases") { val sql1 = "SHOW DATABASES" val sql2 = "SHOW DATABASES LIKE 'defau*'" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 570f68d4592c..22426bf11753 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -540,11 +540,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } else { val oldTableDef = getRawTable(db, withStatsProps.identifier.table) - val (tableSchema, schemaChange) = if (oldTableDef.schema.equals(withStatsProps.schema)) { - (oldTableDef.schema, false) - } else { - (withStatsProps.schema, true) - } + val (tableSchema, schemaChange) = + if (getSchemaFromTableProperties(oldTableDef).equals(withStatsProps.schema)) { + // The table properties keep the original case-sensitiviy for column names + // so we use the schema derived from the old table's table properties for the + // comparison + (oldTableDef.schema, false) + } else { + (withStatsProps.schema, true) + } val newStorage = if (DDLUtils.isHiveTable(tableDefinition)) { tableDefinition.storage From ec57ee9e4a3fc733189e2ad3870b9a215c90bc13 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Mon, 5 Dec 2016 12:31:55 -0800 Subject: [PATCH 05/34] fix testcase --- .../org/apache/spark/sql/hive/HiveExternalCatalog.scala | 5 ++--- .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 22426bf11753..a6f65490b42e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -46,7 +46,6 @@ import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.types.{DataType, StructType} - /** * A persistent implementation of the system catalog using Hive. * All public methods must be synchronized for thread-safety. @@ -411,7 +410,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } } - properties.toMap + properties } private def defaultTablePath(tableIdent: TableIdentifier): String = { @@ -612,7 +611,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat schema = tableSchema, partitionColumnNames = oldTableDef.partitionColumnNames, bucketSpec = oldTableDef.bucketSpec, - properties = newTableProps) + properties = newTableProps.toMap) client.alterTable(newDef) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 04b12407b9c4..d82dbcda4294 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} // TODO(gatorsmile): combine HiveCatalogedDDLSuite and HiveDDLSuite class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeAndAfterEach { From ec74849c2d6181877cad7aa45d7381554065d166 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Wed, 7 Dec 2016 13:56:40 -0800 Subject: [PATCH 06/34] update testcases --- .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index d82dbcda4294..9bd3a59a1dc3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1962,7 +1962,7 @@ class HiveDDLSuite } } - test("alter table add columns -- not datasource table") { + test("alter table add columns -- not support datasource table") { withTempDir { dir => withTable("t_ds") { sql(s"create table t_ds (c1 int, c2 int) using parquet options(path '$dir')") From 8fca889b6f77a4e3ff1a6c6d1323405a613e3ab9 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Wed, 7 Dec 2016 15:10:54 -0800 Subject: [PATCH 07/34] update testcases --- .../spark/sql/execution/command/DDLCommandSuite.scala | 8 ++++++++ 1 file changed, 8 insertions(+) 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 725b88c9ba31..7b508e124b72 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 @@ -807,6 +807,14 @@ class DDLCommandSuite extends PlanTest { """.stripMargin) } + test("alter table: replace columns (not allowed)") { + assertUnsupported( + """ + |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT + |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT + """.stripMargin) + } + test("show databases") { val sql1 = "SHOW DATABASES" val sql2 = "SHOW DATABASES LIKE 'defau*'" From 4a175299a240b499d676bf03e8f48739e4a61fca Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 13 Jan 2017 12:53:44 -0800 Subject: [PATCH 08/34] update testcases --- .../execution/command/DDLCommandSuite.scala | 27 ------------------- 1 file changed, 27 deletions(-) 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 7b508e124b72..13202a57851e 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 @@ -780,33 +780,6 @@ class DDLCommandSuite extends PlanTest { assertUnsupported("ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES") } - test("alter table: add/replace columns (not allowed)") { - assertUnsupported( - """ - |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') - |ADD COLUMNS (new_col1 INT COMMENT 'test_comment', new_col2 LONG - |COMMENT 'test_comment2') CASCADE - """.stripMargin) - assertUnsupported( - """ - |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT - |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT - """.stripMargin) - } - - test("alter table: change column name/type/position/comment (not allowed)") { - assertUnsupported("ALTER TABLE table_name CHANGE col_old_name col_new_name INT") - assertUnsupported( - """ - |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' FIRST CASCADE - """.stripMargin) - assertUnsupported(""" - |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' AFTER column_name RESTRICT - """.stripMargin) - } - test("alter table: replace columns (not allowed)") { assertUnsupported( """ From 96991282c9744219e5526cbcca50547020afd632 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 20 Jan 2017 08:46:36 -0800 Subject: [PATCH 09/34] comments for command caseclass --- .../org/apache/spark/sql/execution/command/tables.scala | 8 ++++++++ 1 file changed, 8 insertions(+) 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 a9249355c4f3..b8b95a43e924 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 @@ -174,6 +174,14 @@ case class AlterTableRenameCommand( } +/** + * A command that add columns to a table + * The syntax of using this command in SQL is: + * {{{ + * ALTER TABLE table_identifier + * ADD COLUMNS (col_name data_type [COMMENT col_comment], ...); + * }}} +*/ case class AlterTableAddColumnsCommand( table: TableIdentifier, columns: Seq[StructField]) extends RunnableCommand { From 9860e5c1725fe0ffc0741ad9b7ccbdb1f069e389 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 20 Jan 2017 16:51:32 -0800 Subject: [PATCH 10/34] udate comments based on review --- .../scala/org/apache/spark/sql/execution/SparkSqlParser.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 52c85a555702..7f49a725c91e 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 @@ -746,7 +746,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * For example: * {{{ - * ALTER TABLE table1 ADD COLUMNS (c1 int); + * ALTER TABLE table1 + * ADD COLUMNS (col_name data_type [COMMENT col_comment], ...); * }}} */ override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) { From dfff364abb645171014340657c11e3e15ea25229 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 3 Feb 2017 12:19:32 -0800 Subject: [PATCH 11/34] SPARK-19261: update to support datasource table and add new testcases --- .../spark/sql/execution/command/ddl.scala | 34 +++++- .../spark/sql/execution/command/tables.scala | 8 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 21 ++++ .../spark/sql/sources/TableScanSuite.scala | 17 +++ .../spark/sql/hive/HiveExternalCatalog.scala | 82 +++++++------- .../sql/hive/execution/HiveDDLSuite.scala | 100 ++++++++++++++---- 6 files changed, 198 insertions(+), 64 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 305a7cd66dcb..e7d11bfde444 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Resolver} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.execution.datasources.PartitioningUtils +import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat, PartitioningUtils} import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration @@ -764,7 +764,9 @@ object DDLUtils { val HIVE_PROVIDER = "hive" def isHiveTable(table: CatalogTable): Boolean = { - table.provider.isDefined && table.provider.get.toLowerCase == HIVE_PROVIDER + // When `CatalogTable` is directly fetched from the catalog, + // CatalogTable.provider = None means the table is a Hive serde table. + !table.provider.isDefined || table.provider.get.toLowerCase == HIVE_PROVIDER } def isDatasourceTable(table: CatalogTable): Boolean = { @@ -818,7 +820,7 @@ object DDLUtils { /** * ALTER TABLE ADD COLUMNS command does not support temporary view/table, - * view, or datasource table yet. + * view, or datasource table with text, orc formats or external provider. */ def verifyAlterTableAddColumn( catalog: SessionCatalog, @@ -833,10 +835,32 @@ object DDLUtils { throw new AnalysisException( s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.") } + if (isDatasourceTable(catalogTable)) { - throw new AnalysisException( - s"${table.toString} is a datasource table, which does not support ALTER ADD COLUMNS yet.") + catalogTable.provider.get match { + case provider if provider.toLowerCase == "text" => + // TextFileFormat can not support adding column either because text datasource table + // is resolved as a single-column table only. + throw new AnalysisException( + s"""${table.toString} is a text format datasource table, + |which does not support ALTER ADD COLUMNS.""".stripMargin) + case provider if provider.toLowerCase == "orc" + || provider.startsWith("org.apache.spark.sql.hive.orc") => + // TODO Current native orc reader can not handle the difference between + // user-specified schema and inferred schema from ORC data file yet. + throw new AnalysisException( + s"""${table.toString} is an ORC datasource table, + |which does not support ALTER ADD COLUMNS.""".stripMargin) + case provider + if (!DataSource.lookupDataSource(provider).newInstance().isInstanceOf[FileFormat]) => + // For datasource table, we only support HadoopFsRelation + throw new AnalysisException( + s"""${table.toString} is a datasource table with external provider, + |which does not support ALTER ADD COLUMNS.""".stripMargin) + case _ => + } } + catalogTable } } 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 b8b95a43e924..955d5b08dbcc 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 @@ -202,9 +202,11 @@ case class AlterTableAddColumnsCommand( // Invalidate the table last, otherwise uncaching the table would load the logical plan // back into the hive metastore cache catalog.refreshTable(table) - - val newSchema = catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns) - catalog.alterTable(catalogTable.copy(schema = newSchema)) + val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length) + val dataData = catalogTable.schema + .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) + catalog.alterTable(catalogTable.copy(schema = + catalogTable.schema.copy(fields = (dataData ++ columns ++ partitionFields).toArray))) Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 5463728ca0c1..7d2f8b1cdd5b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -71,8 +71,20 @@ class JDBCSuite extends SparkFunSuite conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate() conn.prepareStatement( "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() + + conn.prepareStatement("create table test.t_alter_add(c1 int, c2 int)").executeUpdate() + conn.prepareStatement("insert into test.t_alter_add values (1, 2)").executeUpdate() + conn.prepareStatement("insert into test.t_alter_add values (2, 4)").executeUpdate() conn.commit() + sql("DROP TABLE IF EXISTS ds_jdbc") + sql( + s""" + |CREATE TABLE IF NOT EXISTS ds_jdbc + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.T_ALTER_ADD', user 'testUser', password 'testPass') + """.stripMargin.replaceAll("\n", " ")) + sql( s""" |CREATE OR REPLACE TEMPORARY VIEW foobar @@ -970,4 +982,13 @@ class JDBCSuite extends SparkFunSuite assert(sql("select * from people_view").count() == 3) } } + + test("ALTER TABLE ADD COLUMNS") { + conn.prepareStatement("ALTER TABLE test.t_alter_add ADD COLUMN (C3 int)").executeUpdate() + conn.commit() + val e = intercept[AnalysisException] { + sql("ALTER TABLE ds_jdbc ADD COLUMNS (C3 int)") + }.getMessage + assert(e.contains("does not support ALTER ADD COLUMNS")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index b01d15eb917e..6ef99638f339 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -416,4 +416,21 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { val comments = planned.schema.fields.map(_.getComment().getOrElse("NO_COMMENT")).mkString(",") assert(comments === "SN,SA,NO_COMMENT") } + + test("ALTER TABLE ADD COLUMNS does not support RelationProvider") { + withTable("ds_relationProvider") { + sql( + """ + |CREATE TABLE ds_relationProvider + |USING org.apache.spark.sql.sources.SimpleScanSource + |OPTIONS ( + | From '1', + | To '10' + |)""".stripMargin) + val e = intercept[AnalysisException] { + sql("ALTER TABLE ds_relationProvider ADD COLUMNS (c3 int)") + }.getMessage + assert(e.contains("does not support ALTER ADD COLUMNS")) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index a6f65490b42e..a638367a49ea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -22,6 +22,7 @@ import java.lang.reflect.InvocationTargetException import java.util import scala.collection.mutable +import scala.collection.Map import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration @@ -510,15 +511,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat * Note: As of now, this doesn't support altering table schema, partition column names and bucket * specification. We will ignore them even if users do specify different values for these fields. */ - override def alterTable(tableDefinition: CatalogTable): Unit = withClient { - assert(tableDefinition.identifier.database.isDefined) - val db = tableDefinition.identifier.database.get - requireTableExists(db, tableDefinition.identifier.table) - verifyTableProperties(tableDefinition) + override def alterTable(newTableDefinition: CatalogTable): Unit = withClient { + assert(newTableDefinition.identifier.database.isDefined) + val db = newTableDefinition.identifier.database.get + requireTableExists(db, newTableDefinition.identifier.table) + verifyTableProperties(newTableDefinition) // convert table statistics to properties so that we can persist them through hive api - val withStatsProps: CatalogTable = if (tableDefinition.stats.isDefined) { - val stats = tableDefinition.stats.get + val maybeWithStatsPropsTable: CatalogTable = if (newTableDefinition.stats.isDefined) { + val stats = newTableDefinition.stats.get var statsProperties: Map[String, String] = Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) if (stats.rowCount.isDefined) { @@ -529,28 +530,29 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat statsProperties += (columnStatKeyPropName(colName, k) -> v) } } - tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties) + newTableDefinition.copy(properties = newTableDefinition.properties ++ statsProperties) } else { - tableDefinition + newTableDefinition } - if (tableDefinition.tableType == VIEW) { - client.alterTable(withStatsProps) + if (newTableDefinition.tableType == VIEW) { + client.alterTable(maybeWithStatsPropsTable) } else { - val oldTableDef = getRawTable(db, withStatsProps.identifier.table) - - val (tableSchema, schemaChange) = - if (getSchemaFromTableProperties(oldTableDef).equals(withStatsProps.schema)) { - // The table properties keep the original case-sensitiviy for column names - // so we use the schema derived from the old table's table properties for the - // comparison - (oldTableDef.schema, false) + val oldRawTableDef = getRawTable(db, newTableDefinition.identifier.table) + + // restore the table metadata in spark sql format for comparing with the input + // table metadata that is also in spark sql format + val oldRestoredTableDef = restoreTableMetadata(oldRawTableDef) + val (newSchema, schemaChange) = + if (!oldRestoredTableDef.schema.equals(newTableDefinition.schema)) { + (newTableDefinition.schema, true) } else { - (withStatsProps.schema, true) + // maintain the original format of the table schema + (oldRawTableDef.schema, false) } - val newStorage = if (DDLUtils.isHiveTable(tableDefinition)) { - tableDefinition.storage + val newStorage = if (DDLUtils.isHiveTable(newTableDefinition)) { + newTableDefinition.storage } else { // We can't alter the table storage of data source table directly for 2 reasons: // 1. internally we use path option in storage properties to store the value of table @@ -579,38 +581,46 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // want to alter the table location to a file path, we will fail. This should be fixed // in the future. - val newLocation = tableDefinition.storage.locationUri.map(CatalogUtils.URIToString(_)) - val storageWithPathOption = tableDefinition.storage.copy( - properties = tableDefinition.storage.properties ++ newLocation.map("path" -> _)) + val newLocation = newTableDefinition.storage.map(CatalogUtils.URIToString(_)) + val storageWithPathOption = newTableDefinition.storage.copy( + properties = newTableDefinition.storage.properties ++ newLocation.map("path" -> _)) - val oldLocation = getLocationFromStorageProps(oldTableDef) + val oldLocation = getLocationFromStorageProps(oldRawTableDef) if (oldLocation == newLocation) { - storageWithPathOption.copy(locationUri = oldTableDef.storage.locationUri) + storageWithPathOption.copy(locationUri = oldRawTableDef.storage.locationUri) } else { storageWithPathOption } } - val partitionProviderProp = if (tableDefinition.tracksPartitionsInCatalog) { + val partitionProviderProp = if (newTableDefinition.tracksPartitionsInCatalog) { TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_CATALOG } else { TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_FILESYSTEM } - // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition, + // Sets the `partitionColumnNames` and `bucketSpec` from the old table definition, // to retain the spark specific format if it is. Also add old data source properties to table // properties, to retain the data source table format. val dataSourceProps = if (schemaChange) { - tableMetaToTableProps(withStatsProps).filter(_._1.startsWith(DATASOURCE_PREFIX)) + val props = + tableMetaToTableProps(newTableDefinition).filter(_._1.startsWith(DATASOURCE_PREFIX)) + if (newTableDefinition.provider.isDefined + && newTableDefinition.provider.get.toLowerCase != DDLUtils.HIVE_PROVIDER) { + // we only need to populate non-hive provider to the tableprops + props.put(DATASOURCE_PROVIDER, newTableDefinition.provider.get) + } + props } else { - oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) + oldRawTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) } - val newTableProps = dataSourceProps ++ withStatsProps.properties + partitionProviderProp - val newDef = withStatsProps.copy( + val newTableProps = + dataSourceProps ++ maybeWithStatsPropsTable.properties + partitionProviderProp + val newDef = oldRestoredTableDef.copy( storage = newStorage, - schema = tableSchema, - partitionColumnNames = oldTableDef.partitionColumnNames, - bucketSpec = oldTableDef.bucketSpec, + schema = newSchema, + partitionColumnNames = oldRawTableDef.partitionColumnNames, + bucketSpec = oldRawTableDef.bucketSpec, properties = newTableProps.toMap) client.alterTable(newDef) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 9bd3a59a1dc3..1a92e4bd1d6a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1885,14 +1885,13 @@ class HiveDDLSuite } } - test("alter table add columns -- partitioned") { - val tableTypes = Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO") - tableTypes.foreach{ tableType => + Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO").foreach { tableType => + test(s"alter hive serde table add columns -- partitioned - $tableType") { withTable("alter_add_partitioned") { sql( s""" - |CREATE TABLE alter_add_partitioned (c1 int, c2 int) - |PARTITIONED BY (c3 int) STORED AS $tableType + |CREATE TABLE alter_add_partitioned (c1 int, c2 int) + |PARTITIONED BY (c3 int) STORED AS $tableType """.stripMargin) sql("INSERT INTO alter_add_partitioned PARTITION (c3=1) VALUES (1, 2)") @@ -1916,9 +1915,8 @@ class HiveDDLSuite } } - test("alter table add columns -- with predicate") { - val tableTypes = Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO") - tableTypes.foreach { tableType => + Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO").foreach { tableType => + test(s"alter hive serde table add columns -- with predicate - $tableType ") { withTable("alter_add_predicate") { sql(s"CREATE TABLE alter_add_predicate (c1 int, c2 int) STORED AS $tableType") sql("INSERT INTO alter_add_predicate VALUES (1, 2)") @@ -1942,6 +1940,80 @@ class HiveDDLSuite } } + Seq("parquet", "hive", "json", "csv").foreach { provider => + test(s"alter datasource table add columns - $provider") { + withTable("alter_add_ds") { + sql(s"CREATE TABLE alter_add_ds (c1 int) USING $provider") + sql("INSERT INTO alter_add_ds VALUES (1)") + sql("ALTER TABLE alter_add_ds ADD COLUMNS (c2 int)") + checkAnswer( + sql("SELECT * FROM alter_add_ds"), + Seq(Row(1, null)) + ) + checkAnswer( + sql("SELECT * FROM alter_add_ds where c2 is null"), + Seq(Row(1, null)) + ) + + sql("INSERT INTO alter_add_ds VALUES (3, 2)") + checkAnswer( + sql("SELECT * FROM alter_add_ds where c2 = 2"), + Seq(Row(3, 2)) + ) + } + } + } + + Seq("parquet", "hive", "json", "csv").foreach { provider => + test(s"alter datasource table add columns - partitioned - $provider") { + withTable("alter_add_ds") { + sql(s"CREATE TABLE alter_add_ds (c1 int, c2 int) USING $provider partitioned by (c2)") + sql("INSERT INTO alter_add_ds partition(c2 = 2) VALUES (1)") + sql("ALTER TABLE alter_add_ds ADD COLUMNS (c3 int)") + checkAnswer( + sql("SELECT * FROM alter_add_ds"), + Seq(Row(1, null, 2)) + ) + checkAnswer( + sql("SELECT * FROM alter_add_ds where c3 is null"), + Seq(Row(1, null, 2)) + ) + sql("INSERT INTO alter_add_ds partition(c2 =1) VALUES (2, 3)") + checkAnswer( + sql("SELECT * FROM alter_add_ds where c3 = 3"), + Seq(Row(2, 3, 1)) + ) + checkAnswer( + sql("SELECT * FROM alter_add_ds where c2 = 1"), + Seq(Row(2, 3, 1)) + ) + } + } + } + + test("alter datasource table add columns - text format not supported") { + withTable("alter_add_ds_text") { + sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING text") + val e = intercept[AnalysisException] { + sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)") + }.getMessage + assert(e.contains("does not support ALTER ADD COLUMNS")) + } + } + + test("alter datasource table add columns - orc format not supported") { + Seq("orc", "ORC", "org.apache.spark.sql.hive.orc", + "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source => + withTable("alter_add_ds_text") { + sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING $source") + val e = intercept[AnalysisException] { + sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)") + }.getMessage + assert(e.contains("does not support ALTER ADD COLUMNS")) + } + } + } + test("alter table add columns -- not support temp view") { withTempView("tmp_v") { sql("create temporary view tmp_v as select 1 as c1, 2 as c2") @@ -1961,16 +2033,4 @@ class HiveDDLSuite assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) } } - - test("alter table add columns -- not support datasource table") { - withTempDir { dir => - withTable("t_ds") { - sql(s"create table t_ds (c1 int, c2 int) using parquet options(path '$dir')") - val e = intercept[AnalysisException] { - sql("alter table t_ds add columns (c3 int)") - } - assert(e.message.contains("datasource table, which does not support ALTER ADD COLUMNS yet")) - } - } - } } From 9f232546864bd99b8654d85e4798dc462bcc23a6 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 3 Feb 2017 17:15:54 -0800 Subject: [PATCH 12/34] remove workaournd for parquet issue since parquet-1.8.2 is now supported --- .../parquet/SpecificParquetRecordReaderBase.java | 8 +------- .../org/apache/spark/sql/execution/command/tables.scala | 4 ++-- 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index 88da06634d3d..bf8717483575 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -107,13 +107,7 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont footer = readFooter(configuration, file, range(split.getStart(), split.getEnd())); MessageType fileSchema = footer.getFileMetaData().getSchema(); FilterCompat.Filter filter = getFilter(configuration); - try { - blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema); - } catch (IllegalArgumentException e) { - // In the case where a particular parquet files does not contain - // the column(s) in the filter, we don't do filtering at this level - blocks = footer.getBlocks(); - } + blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema); } else { // otherwise we find the row groups that were selected on the client footer = readFooter(configuration, file, NO_FILTER); 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 955d5b08dbcc..fb0c6cedce33 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 @@ -203,10 +203,10 @@ case class AlterTableAddColumnsCommand( // back into the hive metastore cache catalog.refreshTable(table) val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length) - val dataData = catalogTable.schema + val dataSchema = catalogTable.schema .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) catalog.alterTable(catalogTable.copy(schema = - catalogTable.schema.copy(fields = (dataData ++ columns ++ partitionFields).toArray))) + catalogTable.schema.copy(fields = (dataSchema ++ columns ++ partitionFields).toArray))) Seq.empty[Row] } From 180092f038a5c7957633a799c010cda17a2eea60 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Tue, 7 Feb 2017 08:36:46 -0800 Subject: [PATCH 13/34] SPARK-19261: using white list for datasource table types that supports alter add columns --- .../spark/sql/execution/command/ddl.scala | 50 +------------------ .../spark/sql/execution/command/tables.scala | 42 +++++++++++++++- .../sql/hive/execution/HiveDDLSuite.scala | 22 ++++---- 3 files changed, 52 insertions(+), 62 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index e7d11bfde444..0e38a5d511e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -764,9 +764,7 @@ object DDLUtils { val HIVE_PROVIDER = "hive" def isHiveTable(table: CatalogTable): Boolean = { - // When `CatalogTable` is directly fetched from the catalog, - // CatalogTable.provider = None means the table is a Hive serde table. - !table.provider.isDefined || table.provider.get.toLowerCase == HIVE_PROVIDER + table.provider.isDefined && table.provider.get.toLowerCase == HIVE_PROVIDER } def isDatasourceTable(table: CatalogTable): Boolean = { @@ -817,50 +815,4 @@ object DDLUtils { } } } - - /** - * ALTER TABLE ADD COLUMNS command does not support temporary view/table, - * view, or datasource table with text, orc formats or external provider. - */ - def verifyAlterTableAddColumn( - catalog: SessionCatalog, - table: TableIdentifier): CatalogTable = { - if (catalog.isTemporaryTable(table)) { - throw new AnalysisException( - s"${table.toString} is a temporary VIEW, which does not support ALTER ADD COLUMNS.") - } - - val catalogTable = catalog.getTableMetadata(table) - if (catalogTable.tableType == CatalogTableType.VIEW) { - throw new AnalysisException( - s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.") - } - - if (isDatasourceTable(catalogTable)) { - catalogTable.provider.get match { - case provider if provider.toLowerCase == "text" => - // TextFileFormat can not support adding column either because text datasource table - // is resolved as a single-column table only. - throw new AnalysisException( - s"""${table.toString} is a text format datasource table, - |which does not support ALTER ADD COLUMNS.""".stripMargin) - case provider if provider.toLowerCase == "orc" - || provider.startsWith("org.apache.spark.sql.hive.orc") => - // TODO Current native orc reader can not handle the difference between - // user-specified schema and inferred schema from ORC data file yet. - throw new AnalysisException( - s"""${table.toString} is an ORC datasource table, - |which does not support ALTER ADD COLUMNS.""".stripMargin) - case provider - if (!DataSource.lookupDataSource(provider).newInstance().isInstanceOf[FileFormat]) => - // For datasource table, we only support HadoopFsRelation - throw new AnalysisException( - s"""${table.toString} is a datasource table with external provider, - |which does not support ALTER ADD COLUMNS.""".stripMargin) - case _ => - } - } - - catalogTable - } } 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 fb0c6cedce33..2560486cd85d 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 @@ -37,7 +37,10 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.util.quoteIdentifier -import org.apache.spark.sql.execution.datasources.PartitioningUtils +import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat, PartitioningUtils} +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -187,7 +190,7 @@ case class AlterTableAddColumnsCommand( columns: Seq[StructField]) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - val catalogTable = DDLUtils.verifyAlterTableAddColumn(catalog, table) + val catalogTable = verifyAlterTableAddColumn(catalog, table) // 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. @@ -210,6 +213,41 @@ case class AlterTableAddColumnsCommand( Seq.empty[Row] } + + /** + * ALTER TABLE ADD COLUMNS command does not support temporary view/table, + * view, or datasource table with text, orc formats or external provider. + */ + private def verifyAlterTableAddColumn( + catalog: SessionCatalog, + table: TableIdentifier): CatalogTable = { + val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) + + if (catalogTable.tableType == CatalogTableType.VIEW) { + throw new AnalysisException( + s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.") + } + + if (DDLUtils.isDatasourceTable(catalogTable)) { + DataSource.lookupDataSource(catalogTable.provider.get).newInstance() match { + // For datasource table, this command can only support the following File format. + // TextFileFormat only default to one column "value" + // OrcFileFormat can not handle difference between user-specified schema and + // inferred schema yet. TODO, once this issue is resolved , we can add Orc back. + // Hive type is already considered as hive serde table, so the logic will not + // come in here. + case _: JsonFileFormat => + case _: CSVFileFormat => + case _: ParquetFileFormat => + case s => + throw new AnalysisException( + s"""${table.toString} is a datasource table with type $s, + |which does not support ALTER ADD COLUMNS.""".stripMargin) + } + } + + catalogTable + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 1a92e4bd1d6a..25a323202349 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2001,17 +2001,17 @@ class HiveDDLSuite } } - test("alter datasource table add columns - orc format not supported") { - Seq("orc", "ORC", "org.apache.spark.sql.hive.orc", - "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source => - withTable("alter_add_ds_text") { - sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING $source") - val e = intercept[AnalysisException] { - sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)") - }.getMessage - assert(e.contains("does not support ALTER ADD COLUMNS")) - } + Seq("orc", "ORC", "org.apache.spark.sql.hive.orc", + "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source => + test(s"alter datasource table add columns - $source format not supported") { + withTable("alter_add_ds_text") { + sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING $source") + val e = intercept[AnalysisException] { + sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)") + }.getMessage + assert(e.contains("does not support ALTER ADD COLUMNS")) } + } } test("alter table add columns -- not support temp view") { @@ -2020,7 +2020,7 @@ class HiveDDLSuite val e = intercept[AnalysisException] { sql("alter table tmp_v add columns (c3 int)") } - assert(e.message.contains("is a temporary VIEW, which does not support ALTER ADD COLUMNS")) + assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) } } From 5a8aa8045092ceb521f929821179e8ef9d35b080 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Tue, 7 Feb 2017 10:56:04 -0800 Subject: [PATCH 14/34] fix code style --- .../scala/org/apache/spark/sql/execution/command/tables.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 2560486cd85d..5fb4dfa476bc 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 @@ -242,7 +242,8 @@ case class AlterTableAddColumnsCommand( case s => throw new AnalysisException( s"""${table.toString} is a datasource table with type $s, - |which does not support ALTER ADD COLUMNS.""".stripMargin) + |which does not support ALTER ADD COLUMNS. + """.stripMargin) } } From d3860e603ef2b42a19671f59f35ea671b7b51129 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Tue, 7 Feb 2017 11:45:39 -0800 Subject: [PATCH 15/34] fix coding style --- .../scala/org/apache/spark/sql/execution/command/tables.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 5fb4dfa476bc..b3c361542a79 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 @@ -236,9 +236,7 @@ case class AlterTableAddColumnsCommand( // inferred schema yet. TODO, once this issue is resolved , we can add Orc back. // Hive type is already considered as hive serde table, so the logic will not // come in here. - case _: JsonFileFormat => - case _: CSVFileFormat => - case _: ParquetFileFormat => + case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat => case s => throw new AnalysisException( s"""${table.toString} is a datasource table with type $s, From 55577aa14b5cbbc0d159c163c481590d804729eb Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Thu, 23 Feb 2017 23:31:48 -0800 Subject: [PATCH 16/34] update upon review --- .../spark/sql/execution/command/tables.scala | 7 ++-- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 34 ++++++++----------- .../spark/sql/sources/TableScanSuite.scala | 13 +++---- .../spark/sql/hive/HiveExternalCatalog.scala | 32 ++++++++--------- 4 files changed, 40 insertions(+), 46 deletions(-) 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 b3c361542a79..17337b97bd2f 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 @@ -217,10 +217,11 @@ case class AlterTableAddColumnsCommand( /** * ALTER TABLE ADD COLUMNS command does not support temporary view/table, * view, or datasource table with text, orc formats or external provider. + * For datasource table, it currently only supports parquet, json, csv. */ private def verifyAlterTableAddColumn( - catalog: SessionCatalog, - table: TableIdentifier): CatalogTable = { + catalog: SessionCatalog, + table: TableIdentifier): CatalogTable = { val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) if (catalogTable.tableType == CatalogTableType.VIEW) { @@ -239,7 +240,7 @@ case class AlterTableAddColumnsCommand( case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat => case s => throw new AnalysisException( - s"""${table.toString} is a datasource table with type $s, + s"""${table} is a datasource table with type $s, |which does not support ALTER ADD COLUMNS. """.stripMargin) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 7d2f8b1cdd5b..713f7b25fe0a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -72,19 +72,6 @@ class JDBCSuite extends SparkFunSuite conn.prepareStatement( "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() - conn.prepareStatement("create table test.t_alter_add(c1 int, c2 int)").executeUpdate() - conn.prepareStatement("insert into test.t_alter_add values (1, 2)").executeUpdate() - conn.prepareStatement("insert into test.t_alter_add values (2, 4)").executeUpdate() - conn.commit() - - sql("DROP TABLE IF EXISTS ds_jdbc") - sql( - s""" - |CREATE TABLE IF NOT EXISTS ds_jdbc - |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.T_ALTER_ADD', user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) - sql( s""" |CREATE OR REPLACE TEMPORARY VIEW foobar @@ -984,11 +971,20 @@ class JDBCSuite extends SparkFunSuite } test("ALTER TABLE ADD COLUMNS") { - conn.prepareStatement("ALTER TABLE test.t_alter_add ADD COLUMN (C3 int)").executeUpdate() - conn.commit() - val e = intercept[AnalysisException] { - sql("ALTER TABLE ds_jdbc ADD COLUMNS (C3 int)") - }.getMessage - assert(e.contains("does not support ALTER ADD COLUMNS")) + withTable("ds_jdbc") { + sql( + s""" + |CREATE TABLE IF NOT EXISTS ds_jdbc + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass') + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement("ALTER TABLE TEST.PEOPLE ADD COLUMN (C3 int)").executeUpdate() + conn.commit() + val e = intercept[AnalysisException] { + sql("ALTER TABLE ds_jdbc ADD COLUMNS (C3 int)") + }.getMessage + assert(e.contains("does not support ALTER ADD COLUMNS")) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 6ef99638f339..59668b955c26 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -421,12 +421,13 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { withTable("ds_relationProvider") { sql( """ - |CREATE TABLE ds_relationProvider - |USING org.apache.spark.sql.sources.SimpleScanSource - |OPTIONS ( - | From '1', - | To '10' - |)""".stripMargin) + |CREATE TABLE ds_relationProvider + |USING org.apache.spark.sql.sources.SimpleScanSource + |OPTIONS ( + | From '1', + | To '10' + |) + """.stripMargin) val e = intercept[AnalysisException] { sql("ALTER TABLE ds_relationProvider ADD COLUMNS (c3 int)") }.getMessage diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index a638367a49ea..e1391d01534a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -518,7 +518,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat verifyTableProperties(newTableDefinition) // convert table statistics to properties so that we can persist them through hive api - val maybeWithStatsPropsTable: CatalogTable = if (newTableDefinition.stats.isDefined) { + val withStatsProps = if (newTableDefinition.stats.isDefined) { val stats = newTableDefinition.stats.get var statsProperties: Map[String, String] = Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) @@ -536,19 +536,27 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } if (newTableDefinition.tableType == VIEW) { - client.alterTable(maybeWithStatsPropsTable) + client.alterTable(withStatsProps) } else { val oldRawTableDef = getRawTable(db, newTableDefinition.identifier.table) // restore the table metadata in spark sql format for comparing with the input // table metadata that is also in spark sql format val oldRestoredTableDef = restoreTableMetadata(oldRawTableDef) - val (newSchema, schemaChange) = + val (newSchema, dataSourceProps) = if (!oldRestoredTableDef.schema.equals(newTableDefinition.schema)) { - (newTableDefinition.schema, true) + val props = + tableMetaToTableProps(newTableDefinition).filter(_._1.startsWith(DATASOURCE_PREFIX)) + if (newTableDefinition.provider.isDefined && + newTableDefinition.provider.get.toLowerCase != DDLUtils.HIVE_PROVIDER) { + // we only need to populate non-hive provider to the tableprops + props.put(DATASOURCE_PROVIDER, newTableDefinition.provider.get) + } + (newTableDefinition.schema, props) } else { // maintain the original format of the table schema - (oldRawTableDef.schema, false) + (oldRawTableDef.schema, + oldRawTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))) } val newStorage = if (DDLUtils.isHiveTable(newTableDefinition)) { @@ -602,20 +610,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // Sets the `partitionColumnNames` and `bucketSpec` from the old table definition, // to retain the spark specific format if it is. Also add old data source properties to table // properties, to retain the data source table format. - val dataSourceProps = if (schemaChange) { - val props = - tableMetaToTableProps(newTableDefinition).filter(_._1.startsWith(DATASOURCE_PREFIX)) - if (newTableDefinition.provider.isDefined - && newTableDefinition.provider.get.toLowerCase != DDLUtils.HIVE_PROVIDER) { - // we only need to populate non-hive provider to the tableprops - props.put(DATASOURCE_PROVIDER, newTableDefinition.provider.get) - } - props - } else { - oldRawTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) - } val newTableProps = - dataSourceProps ++ maybeWithStatsPropsTable.properties + partitionProviderProp + dataSourceProps ++ withStatsProps.properties + partitionProviderProp val newDef = oldRestoredTableDef.copy( storage = newStorage, schema = newSchema, From 6fa913a2b852407c10e0c3b4f9190bef5ba47fbb Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 24 Feb 2017 16:10:34 -0800 Subject: [PATCH 17/34] refactor code from alterTable function --- .../spark/sql/hive/HiveExternalCatalog.scala | 128 ++++++++++-------- 1 file changed, 69 insertions(+), 59 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index e1391d01534a..a0ebbdb1cee0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -518,22 +518,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat verifyTableProperties(newTableDefinition) // convert table statistics to properties so that we can persist them through hive api - val withStatsProps = if (newTableDefinition.stats.isDefined) { - val stats = newTableDefinition.stats.get - var statsProperties: Map[String, String] = - Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) - if (stats.rowCount.isDefined) { - statsProperties += STATISTICS_NUM_ROWS -> stats.rowCount.get.toString() - } - stats.colStats.foreach { case (colName, colStat) => - colStat.toMap.foreach { case (k, v) => - statsProperties += (columnStatKeyPropName(colName, k) -> v) - } - } - newTableDefinition.copy(properties = newTableDefinition.properties ++ statsProperties) - } else { - newTableDefinition - } + val withStatsProps = populateStatsProps(newTableDefinition) if (newTableDefinition.tableType == VIEW) { client.alterTable(withStatsProps) @@ -543,6 +528,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // restore the table metadata in spark sql format for comparing with the input // table metadata that is also in spark sql format val oldRestoredTableDef = restoreTableMetadata(oldRawTableDef) + val (newSchema, dataSourceProps) = if (!oldRestoredTableDef.schema.equals(newTableDefinition.schema)) { val props = @@ -559,48 +545,6 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat oldRawTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))) } - val newStorage = if (DDLUtils.isHiveTable(newTableDefinition)) { - newTableDefinition.storage - } else { - // We can't alter the table storage of data source table directly for 2 reasons: - // 1. internally we use path option in storage properties to store the value of table - // location, but the given `tableDefinition` is from outside and doesn't have the path - // option, we need to add it manually. - // 2. this data source table may be created on a file, not a directory, then we can't set - // the `locationUri` field and save it to Hive metastore, because Hive only allows - // directory as table location. - // - // For example, an external data source table is created with a single file '/path/to/file'. - // Internally, we will add a path option with value '/path/to/file' to storage properties, - // and set the `locationUri` to a special value due to SPARK-15269(please see - // `saveTableIntoHive` for more details). When users try to get the table metadata back, we - // will restore the `locationUri` field from the path option and remove the path option from - // storage properties. When users try to alter the table storage, the given - // `tableDefinition` will have `locationUri` field with value `/path/to/file` and the path - // option is not set. - // - // Here we need 2 extra steps: - // 1. add path option to storage properties, to match the internal format, i.e. using path - // option to store the value of table location. - // 2. set the `locationUri` field back to the old one from the existing table metadata, - // if users don't want to alter the table location. This step is necessary as the - // `locationUri` is not always same with the path option, e.g. in the above example - // `locationUri` is a special value and we should respect it. Note that, if users - // want to alter the table location to a file path, we will fail. This should be fixed - // in the future. - - val newLocation = newTableDefinition.storage.map(CatalogUtils.URIToString(_)) - val storageWithPathOption = newTableDefinition.storage.copy( - properties = newTableDefinition.storage.properties ++ newLocation.map("path" -> _)) - - val oldLocation = getLocationFromStorageProps(oldRawTableDef) - if (oldLocation == newLocation) { - storageWithPathOption.copy(locationUri = oldRawTableDef.storage.locationUri) - } else { - storageWithPathOption - } - } - val partitionProviderProp = if (newTableDefinition.tracksPartitionsInCatalog) { TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_CATALOG } else { @@ -613,7 +557,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat val newTableProps = dataSourceProps ++ withStatsProps.properties + partitionProviderProp val newDef = oldRestoredTableDef.copy( - storage = newStorage, + storage = newStorageForAlterTable(newTableDefinition, oldRawTableDef), schema = newSchema, partitionColumnNames = oldRawTableDef.partitionColumnNames, bucketSpec = oldRawTableDef.bucketSpec, @@ -642,6 +586,72 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } } + private def newStorageForAlterTable( + newTableDef: CatalogTable, + oldRawTableDef: CatalogTable): CatalogStorageFormat = { + if (DDLUtils.isHiveTable(newTableDef)) { + newTableDef.storage + } else { + // We can't alter the table storage of data source table directly for 2 reasons: + // 1. internally we use path option in storage properties to store the value of table + // location, but the given `tableDefinition` is from outside and doesn't have the path + // option, we need to add it manually. + // 2. this data source table may be created on a file, not a directory, then we can't set + // the `locationUri` field and save it to Hive metastore, because Hive only allows + // directory as table location. + // + // For example, an external data source table is created with a single file '/path/to/file'. + // Internally, we will add a path option with value '/path/to/file' to storage properties, + // and set the `locationUri` to a special value due to SPARK-15269(please see + // `saveTableIntoHive` for more details). When users try to get the table metadata back, we + // will restore the `locationUri` field from the path option and remove the path option from + // storage properties. When users try to alter the table storage, the given + // `tableDefinition` will have `locationUri` field with value `/path/to/file` and the path + // option is not set. + // + // Here we need 2 extra steps: + // 1. add path option to storage properties, to match the internal format, i.e. using path + // option to store the value of table location. + // 2. set the `locationUri` field back to the old one from the existing table metadata, + // if users don't want to alter the table location. This step is necessary as the + // `locationUri` is not always same with the path option, e.g. in the above example + // `locationUri` is a special value and we should respect it. Note that, if users + // want to alter the table location to a file path, we will fail. This should be fixed + // in the future. + + val newLocation = newTableDefinition.storage.map(CatalogUtils.URIToString(_)) + val storageWithPathOption = newTableDefinition.storage.copy( + properties = newTableDefinition.storage.properties ++ newLocation.map("path" -> _)) + + val oldLocation = getLocationFromStorageProps(oldRawTableDef) + if (oldLocation == newLocation) { + storageWithPathOption.copy(locationUri = oldRawTableDef.storage.locationUri) + } else { + storageWithPathOption + } + } + } + + private def populateStatsProps (newTableDefinition: CatalogTable): CatalogTable = { + // convert table statistics to properties so that we can persist them through hive api + if (newTableDefinition.stats.isDefined) { + val stats = newTableDefinition.stats.get + var statsProperties: Map[String, String] = + Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) + if (stats.rowCount.isDefined) { + statsProperties += STATISTICS_NUM_ROWS -> stats.rowCount.get.toString() + } + stats.colStats.foreach { case (colName, colStat) => + colStat.toMap.foreach { case (k, v) => + statsProperties += (columnStatKeyPropName(colName, k) -> v) + } + } + newTableDefinition.copy(properties = newTableDefinition.properties ++ statsProperties) + } else { + newTableDefinition + } + } + override def getTable(db: String, table: String): CatalogTable = withClient { restoreTableMetadata(getRawTable(db, table)) } From 7231efe075bad4723d008da3c21fceffc15140de Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Mon, 6 Mar 2017 11:57:30 -0800 Subject: [PATCH 18/34] rebase and resolve conflict --- .../org/apache/spark/sql/hive/HiveExternalCatalog.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index a0ebbdb1cee0..11df1aa25d4c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -619,9 +619,9 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // want to alter the table location to a file path, we will fail. This should be fixed // in the future. - val newLocation = newTableDefinition.storage.map(CatalogUtils.URIToString(_)) - val storageWithPathOption = newTableDefinition.storage.copy( - properties = newTableDefinition.storage.properties ++ newLocation.map("path" -> _)) + val newLocation = newTableDef.storage.locationUri.map(CatalogUtils.URIToString(_)) + val storageWithPathOption = newTableDef.storage.copy( + properties = newTableDef.storage.properties ++ newLocation.map("path" -> _)) val oldLocation = getLocationFromStorageProps(oldRawTableDef) if (oldLocation == newLocation) { From e4e9ecfcb80db5c4e3840094a4b8566e680d6878 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Thu, 9 Mar 2017 13:06:21 -0800 Subject: [PATCH 19/34] resolve conflicts --- .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 25a323202349..06c962c5c2dd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.types._ // TODO(gatorsmile): combine HiveCatalogedDDLSuite and HiveDDLSuite class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeAndAfterEach { From 75e7441c9a9808784eb94a1a57c047e81d2a4863 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Tue, 14 Mar 2017 14:58:11 -0700 Subject: [PATCH 20/34] using ExternalCatalog.alterTableSchema --- .../sql/catalyst/catalog/SessionCatalog.scala | 20 +++ .../spark/sql/execution/command/tables.scala | 4 +- .../sql/execution/command/DDLSuite.scala | 117 ++++++++++++++++++ .../sql/hive/execution/HiveDDLSuite.scala | 105 ---------------- 4 files changed, 139 insertions(+), 107 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 25aa8d3ba921..9a3c584f52ef 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 @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.types.StructType object SessionCatalog { val DEFAULT_DATABASE = "default" @@ -295,6 +296,25 @@ class SessionCatalog( externalCatalog.alterTable(newTableDefinition) } + /** + * Alter the schema of a table identified by the provided table identifier. The new schema + * should still contain the existing bucket columns and partition columns used by the table. This + * method will also update any Spark SQL-related parameters stored as Hive table properties (such + * as the schema itself). + * + * @param identifier TableIdentifier + * @param newSchema Updated schema to be used for the table (must contain existing partition and + * bucket columns) + */ + def alterTableSchema(identifier: TableIdentifier, newSchema: StructType): Unit = { + val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase)) + val table = formatTableName(identifier.table) + val tableIdentifier = TableIdentifier(table, Some(db)) + requireDbExists(db) + requireTableExists(tableIdentifier) + externalCatalog.alterTableSchema(db, table, newSchema) + } + /** * Return whether a table/view with the specified name exists. If no database is specified, check * with current database. 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 17337b97bd2f..b1a41c83605c 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 @@ -208,8 +208,8 @@ case class AlterTableAddColumnsCommand( val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length) val dataSchema = catalogTable.schema .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) - catalog.alterTable(catalogTable.copy(schema = - catalogTable.schema.copy(fields = (dataSchema ++ columns ++ partitionFields).toArray))) + catalog.alterTableSchema(table, newSchema = + catalogTable.schema.copy(fields = (dataSchema ++ columns ++ partitionFields).toArray)) Seq.empty[Row] } 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 dd76fdde06cd..77fa1af9da89 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 @@ -166,6 +166,42 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo } } + Seq("parquet", "json", "csv").foreach { provider => + test("Alter table add columns") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + withTable("t") { + sql(s"create table t (c1 int) using ${provider}") + sql("insert into table t values (1)") + sql("alter table t add columns (c2 int)") + checkAnswer(sql("select * from t"), Seq(Row(1, null))) + sql("insert into table t values (2, 2)") + checkAnswer(sql("select * from t where c2 is not null"), Seq(Row(2, 2))) + checkAnswer( + sql("select * from t"), + Seq(Row(1, null), Row(2, 2)) + ) + } + } + } + + Seq("parquet", "json", "csv").foreach { provider => + test("Alter table add columns with partitions") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + withTable("t") { + sql(s"create table t (c1 int, c2 int) using ${provider} partitioned by (c2)") + sql("insert into table t values (1, 1)") + sql("alter table t add columns (c3 int)") + checkAnswer(sql("select * from t"), Seq(Row(1, null, 1))) + sql("insert into table t values (2, 2, 3)") + checkAnswer(sql("select * from t where c3 is not null"), Seq(Row(2, 2, 3))) + checkAnswer(sql("select * from t where c2 = 3"), Seq(Row(2, 2, 3))) + checkAnswer( + sql("select * from t"), + Seq(Row(1, null, 1), Row(2, 2, 3)) + ) + } + } + } } abstract class DDLSuite extends QueryTest with SQLTestUtils { @@ -2178,4 +2214,85 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } } + + Seq("parquet", "json", "csv").foreach { provider => + test(s"alter datasource table add columns - $provider") { + withTable("alter_add_ds") { + sql(s"CREATE TABLE alter_add_ds (c1 int) USING $provider") + sql("INSERT INTO alter_add_ds VALUES (1)") + sql("ALTER TABLE alter_add_ds ADD COLUMNS (c2 int)") + checkAnswer( + sql("SELECT * FROM alter_add_ds"), + Seq(Row(1, null)) + ) + checkAnswer( + sql("SELECT * FROM alter_add_ds where c2 is null"), + Seq(Row(1, null)) + ) + + sql("INSERT INTO alter_add_ds VALUES (3, 2)") + checkAnswer( + sql("SELECT * FROM alter_add_ds where c2 = 2"), + Seq(Row(3, 2)) + ) + } + } + } + + Seq("parquet", "json", "csv").foreach { provider => + test(s"alter datasource table add columns - partitioned - $provider") { + withTable("alter_add_ds") { + sql(s"CREATE TABLE alter_add_ds (c1 int, c2 int) USING $provider partitioned by (c2)") + sql("INSERT INTO alter_add_ds partition(c2 = 2) VALUES (1)") + sql("ALTER TABLE alter_add_ds ADD COLUMNS (c3 int)") + checkAnswer( + sql("SELECT * FROM alter_add_ds"), + Seq(Row(1, null, 2)) + ) + checkAnswer( + sql("SELECT * FROM alter_add_ds where c3 is null"), + Seq(Row(1, null, 2)) + ) + sql("INSERT INTO alter_add_ds partition(c2 =1) VALUES (2, 3)") + checkAnswer( + sql("SELECT * FROM alter_add_ds where c3 = 3"), + Seq(Row(2, 3, 1)) + ) + checkAnswer( + sql("SELECT * FROM alter_add_ds where c2 = 1"), + Seq(Row(2, 3, 1)) + ) + } + } + } + + test("alter datasource table add columns - text format not supported") { + withTable("alter_add_ds_text") { + sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING text") + val e = intercept[AnalysisException] { + sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)") + }.getMessage + assert(e.contains("does not support ALTER ADD COLUMNS")) + } + } + + test("alter table add columns -- not support temp view") { + withTempView("tmp_v") { + sql("create temporary view tmp_v as select 1 as c1, 2 as c2") + val e = intercept[AnalysisException] { + sql("alter table tmp_v add columns (c3 int)") + } + assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) + } + } + + test("alter table add columns -- not support view") { + withView("v1") { + sql("create view v1 as select 1 as c1, 2 as c2") + val e = intercept[AnalysisException] { + sql("alter table v1 add columns (c3 int)") + } + assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 06c962c5c2dd..f314a8eb0883 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1861,30 +1861,6 @@ class HiveDDLSuite } } - Seq("a b", "a:b", "a%b").foreach { specialChars => - test(s"location uri contains $specialChars for database") { - try { - withTable("t") { - withTempDir { dir => - val loc = new File(dir, specialChars) - spark.sql(s"CREATE DATABASE tmpdb LOCATION '$loc'") - spark.sql("USE tmpdb") - - Seq(1).toDF("a").write.saveAsTable("t") - val tblloc = new File(loc, "t") - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) - val tblPath = new Path(tblloc.getAbsolutePath) - val fs = tblPath.getFileSystem(spark.sessionState.newHadoopConf()) - assert(table.location == fs.makeQualified(tblPath).toUri) - assert(tblloc.listFiles().nonEmpty) - } - } - } finally { - spark.sql("DROP DATABASE IF EXISTS tmpdb") - } - } - } - Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO").foreach { tableType => test(s"alter hive serde table add columns -- partitioned - $tableType") { withTable("alter_add_partitioned") { @@ -1940,67 +1916,6 @@ class HiveDDLSuite } } - Seq("parquet", "hive", "json", "csv").foreach { provider => - test(s"alter datasource table add columns - $provider") { - withTable("alter_add_ds") { - sql(s"CREATE TABLE alter_add_ds (c1 int) USING $provider") - sql("INSERT INTO alter_add_ds VALUES (1)") - sql("ALTER TABLE alter_add_ds ADD COLUMNS (c2 int)") - checkAnswer( - sql("SELECT * FROM alter_add_ds"), - Seq(Row(1, null)) - ) - checkAnswer( - sql("SELECT * FROM alter_add_ds where c2 is null"), - Seq(Row(1, null)) - ) - - sql("INSERT INTO alter_add_ds VALUES (3, 2)") - checkAnswer( - sql("SELECT * FROM alter_add_ds where c2 = 2"), - Seq(Row(3, 2)) - ) - } - } - } - - Seq("parquet", "hive", "json", "csv").foreach { provider => - test(s"alter datasource table add columns - partitioned - $provider") { - withTable("alter_add_ds") { - sql(s"CREATE TABLE alter_add_ds (c1 int, c2 int) USING $provider partitioned by (c2)") - sql("INSERT INTO alter_add_ds partition(c2 = 2) VALUES (1)") - sql("ALTER TABLE alter_add_ds ADD COLUMNS (c3 int)") - checkAnswer( - sql("SELECT * FROM alter_add_ds"), - Seq(Row(1, null, 2)) - ) - checkAnswer( - sql("SELECT * FROM alter_add_ds where c3 is null"), - Seq(Row(1, null, 2)) - ) - sql("INSERT INTO alter_add_ds partition(c2 =1) VALUES (2, 3)") - checkAnswer( - sql("SELECT * FROM alter_add_ds where c3 = 3"), - Seq(Row(2, 3, 1)) - ) - checkAnswer( - sql("SELECT * FROM alter_add_ds where c2 = 1"), - Seq(Row(2, 3, 1)) - ) - } - } - } - - test("alter datasource table add columns - text format not supported") { - withTable("alter_add_ds_text") { - sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING text") - val e = intercept[AnalysisException] { - sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)") - }.getMessage - assert(e.contains("does not support ALTER ADD COLUMNS")) - } - } - Seq("orc", "ORC", "org.apache.spark.sql.hive.orc", "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source => test(s"alter datasource table add columns - $source format not supported") { @@ -2013,24 +1928,4 @@ class HiveDDLSuite } } } - - test("alter table add columns -- not support temp view") { - withTempView("tmp_v") { - sql("create temporary view tmp_v as select 1 as c1, 2 as c2") - val e = intercept[AnalysisException] { - sql("alter table tmp_v add columns (c3 int)") - } - assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) - } - } - - test("alter table add columns -- not support view") { - withView("v1") { - sql("create view v1 as select 1 as c1, 2 as c2") - val e = intercept[AnalysisException] { - sql("alter table v1 add columns (c3 int)") - } - assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) - } - } } From 9847030b8f9ec8029698c19f3edf41e3cebf5964 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Tue, 14 Mar 2017 23:10:39 -0700 Subject: [PATCH 21/34] add InMemoryCatalog testcases --- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 77fa1af9da89..b99075cb45e4 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 @@ -167,7 +167,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo } Seq("parquet", "json", "csv").foreach { provider => - test("Alter table add columns") { + test(s"Alter table add columns -- ${provider} format") { assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") withTable("t") { sql(s"create table t (c1 int) using ${provider}") @@ -185,7 +185,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo } Seq("parquet", "json", "csv").foreach { provider => - test("Alter table add columns with partitions") { + test(s"Alter table add columns with partitions -- ${provider} format") { assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") withTable("t") { sql(s"create table t (c1 int, c2 int) using ${provider} partitioned by (c2)") From 1a383bb82d9b037d910867d36614d44c24fdbbf3 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Tue, 14 Mar 2017 23:20:59 -0700 Subject: [PATCH 22/34] revert change in HiveExernalCatalog.scala --- .../spark/sql/hive/HiveExternalCatalog.scala | 177 ++++++++---------- 1 file changed, 74 insertions(+), 103 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 11df1aa25d4c..33b21be37203 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -22,7 +22,6 @@ import java.lang.reflect.InvocationTargetException import java.util import scala.collection.mutable -import scala.collection.Map import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration @@ -47,6 +46,7 @@ import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.types.{DataType, StructType} + /** * A persistent implementation of the system catalog using Hive. * All public methods must be synchronized for thread-safety. @@ -511,57 +511,94 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat * Note: As of now, this doesn't support altering table schema, partition column names and bucket * specification. We will ignore them even if users do specify different values for these fields. */ - override def alterTable(newTableDefinition: CatalogTable): Unit = withClient { - assert(newTableDefinition.identifier.database.isDefined) - val db = newTableDefinition.identifier.database.get - requireTableExists(db, newTableDefinition.identifier.table) - verifyTableProperties(newTableDefinition) + override def alterTable(tableDefinition: CatalogTable): Unit = withClient { + assert(tableDefinition.identifier.database.isDefined) + val db = tableDefinition.identifier.database.get + requireTableExists(db, tableDefinition.identifier.table) + verifyTableProperties(tableDefinition) // convert table statistics to properties so that we can persist them through hive api - val withStatsProps = populateStatsProps(newTableDefinition) + val withStatsProps = if (tableDefinition.stats.isDefined) { + val stats = tableDefinition.stats.get + var statsProperties: Map[String, String] = + Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) + if (stats.rowCount.isDefined) { + statsProperties += STATISTICS_NUM_ROWS -> stats.rowCount.get.toString() + } + stats.colStats.foreach { case (colName, colStat) => + colStat.toMap.foreach { case (k, v) => + statsProperties += (columnStatKeyPropName(colName, k) -> v) + } + } + tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties) + } else { + tableDefinition + } - if (newTableDefinition.tableType == VIEW) { + if (tableDefinition.tableType == VIEW) { client.alterTable(withStatsProps) } else { - val oldRawTableDef = getRawTable(db, newTableDefinition.identifier.table) - - // restore the table metadata in spark sql format for comparing with the input - // table metadata that is also in spark sql format - val oldRestoredTableDef = restoreTableMetadata(oldRawTableDef) - - val (newSchema, dataSourceProps) = - if (!oldRestoredTableDef.schema.equals(newTableDefinition.schema)) { - val props = - tableMetaToTableProps(newTableDefinition).filter(_._1.startsWith(DATASOURCE_PREFIX)) - if (newTableDefinition.provider.isDefined && - newTableDefinition.provider.get.toLowerCase != DDLUtils.HIVE_PROVIDER) { - // we only need to populate non-hive provider to the tableprops - props.put(DATASOURCE_PROVIDER, newTableDefinition.provider.get) - } - (newTableDefinition.schema, props) + val oldTableDef = getRawTable(db, withStatsProps.identifier.table) + + val newStorage = if (DDLUtils.isHiveTable(tableDefinition)) { + tableDefinition.storage + } else { + // We can't alter the table storage of data source table directly for 2 reasons: + // 1. internally we use path option in storage properties to store the value of table + // location, but the given `tableDefinition` is from outside and doesn't have the path + // option, we need to add it manually. + // 2. this data source table may be created on a file, not a directory, then we can't set + // the `locationUri` field and save it to Hive metastore, because Hive only allows + // directory as table location. + // + // For example, an external data source table is created with a single file '/path/to/file'. + // Internally, we will add a path option with value '/path/to/file' to storage properties, + // and set the `locationUri` to a special value due to SPARK-15269(please see + // `saveTableIntoHive` for more details). When users try to get the table metadata back, we + // will restore the `locationUri` field from the path option and remove the path option from + // storage properties. When users try to alter the table storage, the given + // `tableDefinition` will have `locationUri` field with value `/path/to/file` and the path + // option is not set. + // + // Here we need 2 extra steps: + // 1. add path option to storage properties, to match the internal format, i.e. using path + // option to store the value of table location. + // 2. set the `locationUri` field back to the old one from the existing table metadata, + // if users don't want to alter the table location. This step is necessary as the + // `locationUri` is not always same with the path option, e.g. in the above example + // `locationUri` is a special value and we should respect it. Note that, if users + // want to alter the table location to a file path, we will fail. This should be fixed + // in the future. + + val newLocation = tableDefinition.storage.locationUri.map(CatalogUtils.URIToString(_)) + val storageWithPathOption = tableDefinition.storage.copy( + properties = tableDefinition.storage.properties ++ newLocation.map("path" -> _)) + + val oldLocation = getLocationFromStorageProps(oldTableDef) + if (oldLocation == newLocation) { + storageWithPathOption.copy(locationUri = oldTableDef.storage.locationUri) } else { - // maintain the original format of the table schema - (oldRawTableDef.schema, - oldRawTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))) + storageWithPathOption } + } - val partitionProviderProp = if (newTableDefinition.tracksPartitionsInCatalog) { + val partitionProviderProp = if (tableDefinition.tracksPartitionsInCatalog) { TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_CATALOG } else { TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_FILESYSTEM } - // Sets the `partitionColumnNames` and `bucketSpec` from the old table definition, + // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition, // to retain the spark specific format if it is. Also add old data source properties to table // properties, to retain the data source table format. - val newTableProps = - dataSourceProps ++ withStatsProps.properties + partitionProviderProp - val newDef = oldRestoredTableDef.copy( - storage = newStorageForAlterTable(newTableDefinition, oldRawTableDef), - schema = newSchema, - partitionColumnNames = oldRawTableDef.partitionColumnNames, - bucketSpec = oldRawTableDef.bucketSpec, - properties = newTableProps.toMap) + val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) + val newTableProps = oldDataSourceProps ++ withStatsProps.properties + partitionProviderProp + val newDef = withStatsProps.copy( + storage = newStorage, + schema = oldTableDef.schema, + partitionColumnNames = oldTableDef.partitionColumnNames, + bucketSpec = oldTableDef.bucketSpec, + properties = newTableProps) client.alterTable(newDef) } @@ -586,72 +623,6 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } } - private def newStorageForAlterTable( - newTableDef: CatalogTable, - oldRawTableDef: CatalogTable): CatalogStorageFormat = { - if (DDLUtils.isHiveTable(newTableDef)) { - newTableDef.storage - } else { - // We can't alter the table storage of data source table directly for 2 reasons: - // 1. internally we use path option in storage properties to store the value of table - // location, but the given `tableDefinition` is from outside and doesn't have the path - // option, we need to add it manually. - // 2. this data source table may be created on a file, not a directory, then we can't set - // the `locationUri` field and save it to Hive metastore, because Hive only allows - // directory as table location. - // - // For example, an external data source table is created with a single file '/path/to/file'. - // Internally, we will add a path option with value '/path/to/file' to storage properties, - // and set the `locationUri` to a special value due to SPARK-15269(please see - // `saveTableIntoHive` for more details). When users try to get the table metadata back, we - // will restore the `locationUri` field from the path option and remove the path option from - // storage properties. When users try to alter the table storage, the given - // `tableDefinition` will have `locationUri` field with value `/path/to/file` and the path - // option is not set. - // - // Here we need 2 extra steps: - // 1. add path option to storage properties, to match the internal format, i.e. using path - // option to store the value of table location. - // 2. set the `locationUri` field back to the old one from the existing table metadata, - // if users don't want to alter the table location. This step is necessary as the - // `locationUri` is not always same with the path option, e.g. in the above example - // `locationUri` is a special value and we should respect it. Note that, if users - // want to alter the table location to a file path, we will fail. This should be fixed - // in the future. - - val newLocation = newTableDef.storage.locationUri.map(CatalogUtils.URIToString(_)) - val storageWithPathOption = newTableDef.storage.copy( - properties = newTableDef.storage.properties ++ newLocation.map("path" -> _)) - - val oldLocation = getLocationFromStorageProps(oldRawTableDef) - if (oldLocation == newLocation) { - storageWithPathOption.copy(locationUri = oldRawTableDef.storage.locationUri) - } else { - storageWithPathOption - } - } - } - - private def populateStatsProps (newTableDefinition: CatalogTable): CatalogTable = { - // convert table statistics to properties so that we can persist them through hive api - if (newTableDefinition.stats.isDefined) { - val stats = newTableDefinition.stats.get - var statsProperties: Map[String, String] = - Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) - if (stats.rowCount.isDefined) { - statsProperties += STATISTICS_NUM_ROWS -> stats.rowCount.get.toString() - } - stats.colStats.foreach { case (colName, colStat) => - colStat.toMap.foreach { case (k, v) => - statsProperties += (columnStatKeyPropName(colName, k) -> v) - } - } - newTableDefinition.copy(properties = newTableDefinition.properties ++ statsProperties) - } else { - newTableDefinition - } - } - override def getTable(db: String, table: String): CatalogTable = withClient { restoreTableMetadata(getRawTable(db, table)) } From f994ce9b2f7a419bc1c5a872f47218057816a937 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Wed, 15 Mar 2017 18:23:07 -0700 Subject: [PATCH 23/34] update upon review --- .../sql/catalyst/catalog/SessionCatalog.scala | 4 +- .../catalog/SessionCatalogSuite.scala | 12 ++++ .../spark/sql/execution/SparkSqlParser.scala | 2 +- .../spark/sql/execution/command/ddl.scala | 2 +- .../spark/sql/execution/command/tables.scala | 4 +- .../sql/execution/command/DDLSuite.scala | 61 ++++--------------- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- 7 files changed, 31 insertions(+), 58 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 9a3c584f52ef..558c43fd1b3c 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 @@ -302,9 +302,9 @@ class SessionCatalog( * method will also update any Spark SQL-related parameters stored as Hive table properties (such * as the schema itself). * - * @param identifier TableIdentifier + * @param identifier TableIdentifier * @param newSchema Updated schema to be used for the table (must contain existing partition and - * bucket columns) + * bucket columns) */ def alterTableSchema(identifier: TableIdentifier, newSchema: StructType): Unit = { val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase)) 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 bb87763e0bbb..6e916cf681cd 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 @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View} +import org.apache.spark.sql.types.IntegerType class InMemorySessionCatalogSuite extends SessionCatalogSuite { protected val utils = new CatalogTestUtils { @@ -450,6 +451,17 @@ abstract class SessionCatalogSuite extends PlanTest { } } + test("alter table add columns") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + sessionCatalog.createTable(newTable("alter_add", "default"), ignoreIfExists = false) + val oldTab = externalCatalog.getTable("default", "alter_add") + sessionCatalog.alterTableSchema(TableIdentifier("alter_add", Some("default")), + oldTab.schema.add("c3", IntegerType)) + val newTab = externalCatalog.getTable("default", "alter_add") + assert(newTab.schema.equals(oldTab.schema.add("c3", IntegerType))) + } + test("get table") { withBasicCatalog { catalog => assert(catalog.getTableMetadata(TableIdentifier("tbl1", Some("db2"))) 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 7f49a725c91e..d4f23f9dd518 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 @@ -753,7 +753,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) { AlterTableAddColumnsCommand( visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.columns).map(visitColTypeList).getOrElse(Nil) + visitColTypeList(ctx.columns) ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 0e38a5d511e1..9d3c55060dfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Resolver} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat, PartitioningUtils} +import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration 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 b1a41c83605c..5103b4e7e4af 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 @@ -240,9 +240,7 @@ case class AlterTableAddColumnsCommand( case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat => case s => throw new AnalysisException( - s"""${table} is a datasource table with type $s, - |which does not support ALTER ADD COLUMNS. - """.stripMargin) + s"Datasource table $table with type $s, which does not support ALTER ADD COLUMNS.") } } 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 b99075cb45e4..7de945cb91cb 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 @@ -165,43 +165,6 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)")) } } - - Seq("parquet", "json", "csv").foreach { provider => - test(s"Alter table add columns -- ${provider} format") { - assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") - withTable("t") { - sql(s"create table t (c1 int) using ${provider}") - sql("insert into table t values (1)") - sql("alter table t add columns (c2 int)") - checkAnswer(sql("select * from t"), Seq(Row(1, null))) - sql("insert into table t values (2, 2)") - checkAnswer(sql("select * from t where c2 is not null"), Seq(Row(2, 2))) - checkAnswer( - sql("select * from t"), - Seq(Row(1, null), Row(2, 2)) - ) - } - } - } - - Seq("parquet", "json", "csv").foreach { provider => - test(s"Alter table add columns with partitions -- ${provider} format") { - assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") - withTable("t") { - sql(s"create table t (c1 int, c2 int) using ${provider} partitioned by (c2)") - sql("insert into table t values (1, 1)") - sql("alter table t add columns (c3 int)") - checkAnswer(sql("select * from t"), Seq(Row(1, null, 1))) - sql("insert into table t values (2, 2, 3)") - checkAnswer(sql("select * from t where c3 is not null"), Seq(Row(2, 2, 3))) - checkAnswer(sql("select * from t where c2 = 3"), Seq(Row(2, 2, 3))) - checkAnswer( - sql("select * from t"), - Seq(Row(1, null, 1), Row(2, 2, 3)) - ) - } - } - } } abstract class DDLSuite extends QueryTest with SQLTestUtils { @@ -2226,13 +2189,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { Seq(Row(1, null)) ) checkAnswer( - sql("SELECT * FROM alter_add_ds where c2 is null"), + sql("SELECT * FROM alter_add_ds WHERE c2 is null"), Seq(Row(1, null)) ) sql("INSERT INTO alter_add_ds VALUES (3, 2)") checkAnswer( - sql("SELECT * FROM alter_add_ds where c2 = 2"), + sql("SELECT * FROM alter_add_ds WHERE c2 = 2"), Seq(Row(3, 2)) ) } @@ -2242,24 +2205,24 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { Seq("parquet", "json", "csv").foreach { provider => test(s"alter datasource table add columns - partitioned - $provider") { withTable("alter_add_ds") { - sql(s"CREATE TABLE alter_add_ds (c1 int, c2 int) USING $provider partitioned by (c2)") - sql("INSERT INTO alter_add_ds partition(c2 = 2) VALUES (1)") + sql(s"CREATE TABLE alter_add_ds (c1 int, c2 int) USING $provider PARTITIONED BY (c2)") + sql("INSERT INTO alter_add_ds PARTITION(c2 = 2) VALUES (1)") sql("ALTER TABLE alter_add_ds ADD COLUMNS (c3 int)") checkAnswer( sql("SELECT * FROM alter_add_ds"), Seq(Row(1, null, 2)) ) checkAnswer( - sql("SELECT * FROM alter_add_ds where c3 is null"), + sql("SELECT * FROM alter_add_ds WHERE c3 is null"), Seq(Row(1, null, 2)) ) - sql("INSERT INTO alter_add_ds partition(c2 =1) VALUES (2, 3)") + sql("INSERT INTO alter_add_ds PARTITION(c2 =1) VALUES (2, 3)") checkAnswer( - sql("SELECT * FROM alter_add_ds where c3 = 3"), + sql("SELECT * FROM alter_add_ds WHERE c3 = 3"), Seq(Row(2, 3, 1)) ) checkAnswer( - sql("SELECT * FROM alter_add_ds where c2 = 1"), + sql("SELECT * FROM alter_add_ds WHERE c2 = 1"), Seq(Row(2, 3, 1)) ) } @@ -2278,9 +2241,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { test("alter table add columns -- not support temp view") { withTempView("tmp_v") { - sql("create temporary view tmp_v as select 1 as c1, 2 as c2") + sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2") val e = intercept[AnalysisException] { - sql("alter table tmp_v add columns (c3 int)") + sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)") } assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) } @@ -2288,9 +2251,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { test("alter table add columns -- not support view") { withView("v1") { - sql("create view v1 as select 1 as c1, 2 as c2") + sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2") val e = intercept[AnalysisException] { - sql("alter table v1 add columns (c3 int)") + sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)") } assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 713f7b25fe0a..05bbfa693401 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -964,7 +964,7 @@ class JDBCSuite extends SparkFunSuite |CREATE TEMPORARY VIEW people_view |USING org.apache.spark.sql.jdbc |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin) assert(sql("select * from people_view").count() == 3) } @@ -977,7 +977,7 @@ class JDBCSuite extends SparkFunSuite |CREATE TABLE IF NOT EXISTS ds_jdbc |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin) conn.prepareStatement("ALTER TABLE TEST.PEOPLE ADD COLUMN (C3 int)").executeUpdate() conn.commit() From 5bf7360834e257a2e0083a5f92f24da73416780d Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Thu, 16 Mar 2017 00:45:47 -0700 Subject: [PATCH 24/34] add checking for duplicate column names --- .../catalog/SessionCatalogSuite.scala | 8 +-- .../spark/sql/execution/command/tables.scala | 33 +++++++---- .../sql/execution/command/DDLSuite.scala | 50 +++++++++------- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 1 + .../sql/hive/execution/HiveDDLSuite.scala | 57 +++++++++++-------- 5 files changed, 90 insertions(+), 59 deletions(-) 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 6e916cf681cd..aa0304621f64 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 @@ -454,11 +454,11 @@ abstract class SessionCatalogSuite extends PlanTest { test("alter table add columns") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - sessionCatalog.createTable(newTable("alter_add", "default"), ignoreIfExists = false) - val oldTab = externalCatalog.getTable("default", "alter_add") - sessionCatalog.alterTableSchema(TableIdentifier("alter_add", Some("default")), + sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false) + val oldTab = externalCatalog.getTable("default", "t1") + sessionCatalog.alterTableSchema(TableIdentifier("t1", Some("default")), oldTab.schema.add("c3", IntegerType)) - val newTab = externalCatalog.getTable("default", "alter_add") + val newTab = externalCatalog.getTable("default", "t1") assert(newTab.schema.equals(oldTab.schema.add("c3", IntegerType))) } 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 5103b4e7e4af..aa83c78da9e6 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 @@ -192,28 +192,37 @@ case class AlterTableAddColumnsCommand( val catalog = sparkSession.sessionState.catalog val catalogTable = verifyAlterTableAddColumn(catalog, table) - // 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(table.unquotedString)).getOrElse(false) - if (wasCached) { - try { - sparkSession.catalog.uncacheTable(table.unquotedString) - } catch { - case NonFatal(e) => log.warn(e.toString, e) - } + try { + sparkSession.catalog.uncacheTable(table.unquotedString) + } catch { + case NonFatal(e) => + log.warn(s"Exception when attempting to uncache table ${table.unquotedString}", e) } + // Invalidate the table last, otherwise uncaching the table would load the logical plan // back into the hive metastore cache catalog.refreshTable(table) val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length) - val dataSchema = catalogTable.schema - .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) + val newSchemaFields = catalogTable.schema + .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) ++ + columns ++ partitionFields + checkDuplication(newSchemaFields.map(_.name)) catalog.alterTableSchema(table, newSchema = - catalogTable.schema.copy(fields = (dataSchema ++ columns ++ partitionFields).toArray)) + catalogTable.schema.copy(fields = newSchemaFields.toArray)) Seq.empty[Row] } + private def checkDuplication(colNames: Seq[String]): Unit = { + if (colNames.distinct.length != colNames.length) { + val duplicateColumns = colNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => x + } + throw new AnalysisException( + s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}") + } + } + /** * ALTER TABLE ADD COLUMNS command does not support temporary view/table, * view, or datasource table with text, orc formats or external provider. 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 7de945cb91cb..fc5b9f5795e2 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 @@ -2180,22 +2180,22 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { Seq("parquet", "json", "csv").foreach { provider => test(s"alter datasource table add columns - $provider") { - withTable("alter_add_ds") { - sql(s"CREATE TABLE alter_add_ds (c1 int) USING $provider") - sql("INSERT INTO alter_add_ds VALUES (1)") - sql("ALTER TABLE alter_add_ds ADD COLUMNS (c2 int)") + withTable("t1") { + sql(s"CREATE TABLE t1 (c1 int) USING $provider") + sql("INSERT INTO t1 VALUES (1)") + sql("ALTER TABLE t1 ADD COLUMNS (c2 int)") checkAnswer( - sql("SELECT * FROM alter_add_ds"), + sql("SELECT * FROM t1"), Seq(Row(1, null)) ) checkAnswer( - sql("SELECT * FROM alter_add_ds WHERE c2 is null"), + sql("SELECT * FROM t1 WHERE c2 is null"), Seq(Row(1, null)) ) - sql("INSERT INTO alter_add_ds VALUES (3, 2)") + sql("INSERT INTO t1 VALUES (3, 2)") checkAnswer( - sql("SELECT * FROM alter_add_ds WHERE c2 = 2"), + sql("SELECT * FROM t1 WHERE c2 = 2"), Seq(Row(3, 2)) ) } @@ -2204,25 +2204,25 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { Seq("parquet", "json", "csv").foreach { provider => test(s"alter datasource table add columns - partitioned - $provider") { - withTable("alter_add_ds") { - sql(s"CREATE TABLE alter_add_ds (c1 int, c2 int) USING $provider PARTITIONED BY (c2)") - sql("INSERT INTO alter_add_ds PARTITION(c2 = 2) VALUES (1)") - sql("ALTER TABLE alter_add_ds ADD COLUMNS (c3 int)") + withTable("t1") { + sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)") + sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)") + sql("ALTER TABLE t1 ADD COLUMNS (c3 int)") checkAnswer( - sql("SELECT * FROM alter_add_ds"), + sql("SELECT * FROM t1"), Seq(Row(1, null, 2)) ) checkAnswer( - sql("SELECT * FROM alter_add_ds WHERE c3 is null"), + sql("SELECT * FROM t1 WHERE c3 is null"), Seq(Row(1, null, 2)) ) - sql("INSERT INTO alter_add_ds PARTITION(c2 =1) VALUES (2, 3)") + sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)") checkAnswer( - sql("SELECT * FROM alter_add_ds WHERE c3 = 3"), + sql("SELECT * FROM t1 WHERE c3 = 3"), Seq(Row(2, 3, 1)) ) checkAnswer( - sql("SELECT * FROM alter_add_ds WHERE c2 = 1"), + sql("SELECT * FROM t1 WHERE c2 = 1"), Seq(Row(2, 3, 1)) ) } @@ -2230,10 +2230,10 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } test("alter datasource table add columns - text format not supported") { - withTable("alter_add_ds_text") { - sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING text") + withTable("t1") { + sql(s"CREATE TABLE t1 (c1 int) USING text") val e = intercept[AnalysisException] { - sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)") + sql("ALTER TABLE t1 ADD COLUMNS (c2 int)") }.getMessage assert(e.contains("does not support ALTER ADD COLUMNS")) } @@ -2258,4 +2258,14 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) } } + + test("alter table add columns with existing column name") { + withTable("t1") { + sql(s"CREATE TABLE t1 (c1 int) USING PARQUET") + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (c1 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 05bbfa693401..e6998de7279d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -71,6 +71,7 @@ class JDBCSuite extends SparkFunSuite conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate() conn.prepareStatement( "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() + conn.commit() sql( s""" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index f314a8eb0883..b8ccd5f7fca5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -112,6 +112,7 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { import testImplicits._ + val hiveFormats = Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO") override def afterEach(): Unit = { try { @@ -1861,55 +1862,55 @@ class HiveDDLSuite } } - Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO").foreach { tableType => + hiveFormats.foreach { tableType => test(s"alter hive serde table add columns -- partitioned - $tableType") { - withTable("alter_add_partitioned") { + withTable("tab") { sql( s""" - |CREATE TABLE alter_add_partitioned (c1 int, c2 int) + |CREATE TABLE tab (c1 int, c2 int) |PARTITIONED BY (c3 int) STORED AS $tableType """.stripMargin) - sql("INSERT INTO alter_add_partitioned PARTITION (c3=1) VALUES (1, 2)") - sql("ALTER TABLE alter_add_partitioned ADD COLUMNS (c4 int)") + sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)") + sql("ALTER TABLE tab ADD COLUMNS (c4 int)") checkAnswer( - sql("SELECT * FROM alter_add_partitioned WHERE c3 = 1"), + sql("SELECT * FROM tab WHERE c3 = 1"), Seq(Row(1, 2, null, 1)) ) - assert(sql("SELECT * FROM alter_add_partitioned").schema + assert(sql("SELECT * FROM tab").schema .contains(StructField("c4", IntegerType))) - sql("INSERT INTO alter_add_partitioned PARTITION (c3=2) VALUES (2, 3, 4)") + sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)") checkAnswer( - sql("SELECT * FROM alter_add_partitioned"), + sql("SELECT * FROM tab"), Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2)) ) checkAnswer( - sql("SELECT * FROM alter_add_partitioned WHERE c3 = 2 AND c4 IS NOT NULL"), + sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"), Seq(Row(2, 3, 4, 2)) ) } } } - Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO").foreach { tableType => + hiveFormats.foreach { tableType => test(s"alter hive serde table add columns -- with predicate - $tableType ") { - withTable("alter_add_predicate") { - sql(s"CREATE TABLE alter_add_predicate (c1 int, c2 int) STORED AS $tableType") - sql("INSERT INTO alter_add_predicate VALUES (1, 2)") - sql("ALTER TABLE alter_add_predicate ADD COLUMNS (c4 int)") + withTable("tab") { + sql(s"CREATE TABLE tab (c1 int, c2 int) STORED AS $tableType") + sql("INSERT INTO tab VALUES (1, 2)") + sql("ALTER TABLE tab ADD COLUMNS (c4 int)") checkAnswer( - sql("SELECT * FROM alter_add_predicate WHERE c4 IS NULL"), + sql("SELECT * FROM tab WHERE c4 IS NULL"), Seq(Row(1, 2, null)) ) - assert(sql("SELECT * FROM alter_add_predicate").schema + assert(sql("SELECT * FROM tab").schema .contains(StructField("c4", IntegerType))) - sql("INSERT INTO alter_add_predicate VALUES (2, 3, 4)") + sql("INSERT INTO tab VALUES (2, 3, 4)") checkAnswer( - sql("SELECT * FROM alter_add_predicate WHERE c4 = 4 "), + sql("SELECT * FROM tab WHERE c4 = 4 "), Seq(Row(2, 3, 4)) ) checkAnswer( - sql("SELECT * FROM alter_add_predicate"), + sql("SELECT * FROM tab"), Seq(Row(1, 2, null), Row(2, 3, 4)) ) } @@ -1919,13 +1920,23 @@ class HiveDDLSuite Seq("orc", "ORC", "org.apache.spark.sql.hive.orc", "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source => test(s"alter datasource table add columns - $source format not supported") { - withTable("alter_add_ds_text") { - sql(s"CREATE TABLE alter_add_ds_text (c1 int) USING $source") + withTable("tab") { + sql(s"CREATE TABLE tab (c1 int) USING $source") val e = intercept[AnalysisException] { - sql("ALTER TABLE alter_add_ds_text ADD COLUMNS (c2 int)") + sql("ALTER TABLE tab ADD COLUMNS (c2 int)") }.getMessage assert(e.contains("does not support ALTER ADD COLUMNS")) } } } + + test("alter table add columns with existing partition column name") { + withTable("tab") { + sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET") + val e = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (c2 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } + } } From 599c45e6fb4febcda5cb74b972e14fe232c334bd Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Thu, 16 Mar 2017 11:31:59 -0700 Subject: [PATCH 25/34] add case sensativity for duplicate name checking and new testcases --- .../spark/sql/execution/command/tables.scala | 27 +++++++++---- .../sql/execution/command/DDLSuite.scala | 40 +++++++++++++++++-- .../sql/hive/execution/HiveDDLSuite.scala | 3 +- 3 files changed, 57 insertions(+), 13 deletions(-) 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 aa83c78da9e6..e38a0b9f8429 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 @@ -193,10 +193,10 @@ case class AlterTableAddColumnsCommand( val catalogTable = verifyAlterTableAddColumn(catalog, table) try { - sparkSession.catalog.uncacheTable(table.unquotedString) + sparkSession.catalog.uncacheTable(table.quotedString) } catch { case NonFatal(e) => - log.warn(s"Exception when attempting to uncache table ${table.unquotedString}", e) + log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e) } // Invalidate the table last, otherwise uncaching the table would load the logical plan @@ -206,16 +206,21 @@ case class AlterTableAddColumnsCommand( val newSchemaFields = catalogTable.schema .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) ++ columns ++ partitionFields - checkDuplication(newSchemaFields.map(_.name)) + checkDuplication(sparkSession, newSchemaFields) catalog.alterTableSchema(table, newSchema = catalogTable.schema.copy(fields = newSchemaFields.toArray)) Seq.empty[Row] } - private def checkDuplication(colNames: Seq[String]): Unit = { - if (colNames.distinct.length != colNames.length) { - val duplicateColumns = colNames.groupBy(identity).collect { + private def checkDuplication(sparkSession: SparkSession, fields: Seq[StructField]): Unit = { + val columnNames = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { + fields.map(_.name) + } else { + fields.map(_.name.toLowerCase) + } + if (columnNames.distinct.length != columnNames.length) { + val duplicateColumns = columnNames.groupBy(identity).collect { case (x, ys) if ys.length > 1 => x } throw new AnalysisException( @@ -235,7 +240,10 @@ case class AlterTableAddColumnsCommand( if (catalogTable.tableType == CatalogTableType.VIEW) { throw new AnalysisException( - s"${table.toString} is a VIEW, which does not support ALTER ADD COLUMNS.") + s""" + |ALTER ADD COLUMNS does not support views. + |You must drop and re-create the views for adding the new columns. Views: $table + """.stripMargin) } if (DDLUtils.isDatasourceTable(catalogTable)) { @@ -249,7 +257,10 @@ case class AlterTableAddColumnsCommand( case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat => case s => throw new AnalysisException( - s"Datasource table $table with type $s, which does not support ALTER ADD COLUMNS.") + s""" + |ALTER ADD COLUMNS does not support datasource table with type $s. + |You must drop and re-create the views for adding the new columns. Tables: $table + """.stripMargin) } } 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 fc5b9f5795e2..5be02e08cc2e 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 @@ -2235,7 +2235,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("ALTER TABLE t1 ADD COLUMNS (c2 int)") }.getMessage - assert(e.contains("does not support ALTER ADD COLUMNS")) + assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type")) } } @@ -2245,7 +2245,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)") } - assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) + assert(e.message.contains("ALTER ADD COLUMNS does not support views")) } } @@ -2255,17 +2255,49 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)") } - assert(e.message.contains("is a VIEW, which does not support ALTER ADD COLUMNS")) + assert(e.message.contains("ALTER ADD COLUMNS does not support views")) } } test("alter table add columns with existing column name") { withTable("t1") { - sql(s"CREATE TABLE t1 (c1 int) USING PARQUET") + sql("CREATE TABLE t1 (c1 int) USING PARQUET") val e = intercept[AnalysisException] { sql("ALTER TABLE t1 ADD COLUMNS (c1 string)") }.getMessage assert(e.contains("Found duplicate column(s)")) } } + + Seq("true", "false").foreach { caseSensitive => + test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { + withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { + withTable("t1") { + sql("CREATE TABLE t1 (c1 int) USING PARQUET") + if (caseSensitive == "false") { + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } else { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + assert(sql("SELECT * FROM t1").schema + .equals(new StructType().add("c1", IntegerType).add("C1", StringType))) + } + } + } + } + } + + test("alter table add columns to table referenced by a view") { + withTable("t1") { + withView("v1") { + sql("CREATE TABLE t1 (c1 int, c2 int) USING PARQUET") + sql("CREATE VIEW v1 AS SELECT * FROM t1") + val originViewSchema = sql("SELECT * FROM v1").schema + sql("ALTER TABLE t1 ADD COLUMNS (c3 int)") + assert(sql("SELECT * FROM v1").schema == originViewSchema) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index b8ccd5f7fca5..7b1add011f10 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1925,7 +1925,8 @@ class HiveDDLSuite val e = intercept[AnalysisException] { sql("ALTER TABLE tab ADD COLUMNS (c2 int)") }.getMessage - assert(e.contains("does not support ALTER ADD COLUMNS")) + assert( + e.contains(s"ALTER ADD COLUMNS does not support datasource table with type")) } } } From b3edfea32943c014cf62a8da273b941a58556b20 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Thu, 16 Mar 2017 11:35:51 -0700 Subject: [PATCH 26/34] typo --- .../scala/org/apache/spark/sql/execution/command/tables.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e38a0b9f8429..05d6af1e4f11 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 @@ -259,7 +259,7 @@ case class AlterTableAddColumnsCommand( throw new AnalysisException( s""" |ALTER ADD COLUMNS does not support datasource table with type $s. - |You must drop and re-create the views for adding the new columns. Tables: $table + |You must drop and re-create the table for adding the new columns. Tables: $table """.stripMargin) } } From 7d8a515b8327d8aec7340cc4d185bf7b2ec8c9be Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 17 Mar 2017 00:14:37 -0700 Subject: [PATCH 27/34] resolve conflicts and modify testcases --- .../catalog/SessionCatalogSuite.scala | 27 ++++++++---- .../sql/execution/command/DDLSuite.scala | 42 ++++++++++--------- .../sql/hive/execution/HiveDDLSuite.scala | 21 ++++++++++ 3 files changed, 61 insertions(+), 29 deletions(-) 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 aa0304621f64..f0809218503d 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View} -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class InMemorySessionCatalogSuite extends SessionCatalogSuite { protected val utils = new CatalogTestUtils { @@ -452,14 +452,23 @@ abstract class SessionCatalogSuite extends PlanTest { } test("alter table add columns") { - val externalCatalog = newBasicCatalog() - val sessionCatalog = new SessionCatalog(externalCatalog) - sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false) - val oldTab = externalCatalog.getTable("default", "t1") - sessionCatalog.alterTableSchema(TableIdentifier("t1", Some("default")), - oldTab.schema.add("c3", IntegerType)) - val newTab = externalCatalog.getTable("default", "t1") - assert(newTab.schema.equals(oldTab.schema.add("c3", IntegerType))) + withBasicCatalog { sessionCatalog => + sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false) + val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1") + sessionCatalog.alterTableSchema(TableIdentifier("t1", Some("default")), + oldTab.schema.add("c3", IntegerType)) + val newTab = sessionCatalog.externalCatalog.getTable("default", "t1") + if (sessionCatalog.externalCatalog.isInstanceOf[InMemoryCatalog]) { + assert(newTab.schema.toString == oldTab.schema.add("c3", IntegerType).toString) + } else { + // HiveExternalCatalog will always arrange the partition columns to the end + val oldTabSchema = StructType(oldTab.schema.take( + oldTab.schema.length - oldTab.partitionColumnNames.length) ++ + Seq(StructField("c3", IntegerType)) ++ + oldTab.schema.takeRight(oldTab.partitionColumnNames.length)) + assert(newTab.schema.toString == oldTabSchema.toString) + } + } } test("get table") { 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 5be02e08cc2e..e803fec74719 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 @@ -165,6 +165,28 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)")) } } + + Seq("true", "false").foreach { caseSensitive => + test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { + withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { + withTable("t1") { + sql("CREATE TABLE t1 (c1 int) USING PARQUET") + if (caseSensitive == "false") { + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } else { + // hive catalog will still complains that c1 is duplicate column name because hive + // identifiers are case insensitive. + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + assert(sql("SELECT * FROM t1").schema + .equals(new StructType().add("c1", IntegerType).add("C1", StringType))) + } + } + } + } + } } abstract class DDLSuite extends QueryTest with SQLTestUtils { @@ -2269,26 +2291,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - Seq("true", "false").foreach { caseSensitive => - test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { - withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { - withTable("t1") { - sql("CREATE TABLE t1 (c1 int) USING PARQUET") - if (caseSensitive == "false") { - val e = intercept[AnalysisException] { - sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") - }.getMessage - assert(e.contains("Found duplicate column(s)")) - } else { - sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") - assert(sql("SELECT * FROM t1").schema - .equals(new StructType().add("c1", IntegerType).add("C1", StringType))) - } - } - } - } - } - test("alter table add columns to table referenced by a view") { withTable("t1") { withView("v1") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 7b1add011f10..20ded6c73ad4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1940,4 +1940,25 @@ class HiveDDLSuite assert(e.contains("Found duplicate column(s)")) } } + + Seq("true", "false").foreach { caseSensitive => + test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { + withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { + withTable("t1") { + sql("CREATE TABLE t1 (c1 int) USING PARQUET") + if (caseSensitive == "false") { + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } else { + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + }.getMessage + assert(e.contains("HiveException")) + } + } + } + } + } } From e895278f2b4a539334e2e366aac4e78547b198c4 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 17 Mar 2017 01:04:21 -0700 Subject: [PATCH 28/34] update testcases --- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 2 -- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- .../org/apache/spark/sql/sources/TableScanSuite.scala | 8 ++++---- .../apache/spark/sql/hive/execution/HiveDDLSuite.scala | 2 ++ 4 files changed, 7 insertions(+), 7 deletions(-) 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 e803fec74719..a9366e7ddeb7 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 @@ -177,8 +177,6 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo }.getMessage assert(e.contains("Found duplicate column(s)")) } else { - // hive catalog will still complains that c1 is duplicate column name because hive - // identifiers are case insensitive. sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") assert(sql("SELECT * FROM t1").schema .equals(new StructType().add("c1", IntegerType).add("C1", StringType))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index e6998de7279d..09307319b81c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -985,7 +985,7 @@ class JDBCSuite extends SparkFunSuite val e = intercept[AnalysisException] { sql("ALTER TABLE ds_jdbc ADD COLUMNS (C3 int)") }.getMessage - assert(e.contains("does not support ALTER ADD COLUMNS")) + assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 59668b955c26..2ffd490f93b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -418,10 +418,10 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { } test("ALTER TABLE ADD COLUMNS does not support RelationProvider") { - withTable("ds_relationProvider") { + withTable("tab") { sql( """ - |CREATE TABLE ds_relationProvider + |CREATE TABLE tab |USING org.apache.spark.sql.sources.SimpleScanSource |OPTIONS ( | From '1', @@ -429,9 +429,9 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { |) """.stripMargin) val e = intercept[AnalysisException] { - sql("ALTER TABLE ds_relationProvider ADD COLUMNS (c3 int)") + sql("ALTER TABLE tab ADD COLUMNS (c3 int)") }.getMessage - assert(e.contains("does not support ALTER ADD COLUMNS")) + assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 20ded6c73ad4..33c2d5a8bfd6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1952,6 +1952,8 @@ class HiveDDLSuite }.getMessage assert(e.contains("Found duplicate column(s)")) } else { + // hive catalog will still complains that c1 is duplicate column name because hive + // identifiers are case insensitive. val e = intercept[AnalysisException] { sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") }.getMessage From e171ac402ce2835eaaf73b98a6b1b21f56aba57b Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 17 Mar 2017 15:13:06 -0700 Subject: [PATCH 29/34] move checkduplicate and schema arrangement to SessionCatalog.alterTableAddColumns --- .../sql/catalyst/catalog/SessionCatalog.scala | 40 ++++++++++++++----- .../catalog/SessionCatalogSuite.scala | 22 +++++----- .../spark/sql/execution/command/tables.scala | 30 ++------------ 3 files changed, 43 insertions(+), 49 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 558c43fd1b3c..a72eb24b76e4 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 @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.StringUtils -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{StructField, StructType} object SessionCatalog { val DEFAULT_DATABASE = "default" @@ -297,22 +297,42 @@ class SessionCatalog( } /** - * Alter the schema of a table identified by the provided table identifier. The new schema - * should still contain the existing bucket columns and partition columns used by the table. This - * method will also update any Spark SQL-related parameters stored as Hive table properties (such - * as the schema itself). - * + * Alter the schema of a table identified by the provided table identifier to add new columns * @param identifier TableIdentifier - * @param newSchema Updated schema to be used for the table (must contain existing partition and - * bucket columns) + * @param columns new columns + * @param caseSensitive enforce case sensitivity for column names */ - def alterTableSchema(identifier: TableIdentifier, newSchema: StructType): Unit = { + def alterTableAddColumns( + identifier: TableIdentifier, + columns: Seq[StructField], + caseSensitive: Boolean): Unit = { val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase)) val table = formatTableName(identifier.table) val tableIdentifier = TableIdentifier(table, Some(db)) requireDbExists(db) requireTableExists(tableIdentifier) - externalCatalog.alterTableSchema(db, table, newSchema) + + val catalogTable = externalCatalog.getTable(db, table) + val partitionSchema = catalogTable.partitionSchema + // reorder schema columns w.r.t partition columns + val newSchemaFields = catalogTable.dataSchema.fields ++ columns ++ partitionSchema.fields + checkDuplication(newSchemaFields, caseSensitive) + externalCatalog.alterTableSchema(db, table, catalogTable.schema.copy(fields = newSchemaFields)) + } + + private def checkDuplication(fields: Seq[StructField], caseSensitive: Boolean): Unit = { + val columnNames = if (caseSensitive) { + fields.map(_.name) + } else { + fields.map(_.name.toLowerCase) + } + if (columnNames.distinct.length != columnNames.length) { + val duplicateColumns = columnNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => x + } + throw new AnalysisException( + s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}") + } } /** 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 f0809218503d..fd565d49e39c 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View} -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.types._ class InMemorySessionCatalogSuite extends SessionCatalogSuite { protected val utils = new CatalogTestUtils { @@ -455,19 +455,15 @@ abstract class SessionCatalogSuite extends PlanTest { withBasicCatalog { sessionCatalog => sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false) val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1") - sessionCatalog.alterTableSchema(TableIdentifier("t1", Some("default")), - oldTab.schema.add("c3", IntegerType)) + sessionCatalog.alterTableAddColumns( + TableIdentifier("t1", Some("default")), + Seq(StructField("c3", IntegerType)), false) + val newTab = sessionCatalog.externalCatalog.getTable("default", "t1") - if (sessionCatalog.externalCatalog.isInstanceOf[InMemoryCatalog]) { - assert(newTab.schema.toString == oldTab.schema.add("c3", IntegerType).toString) - } else { - // HiveExternalCatalog will always arrange the partition columns to the end - val oldTabSchema = StructType(oldTab.schema.take( - oldTab.schema.length - oldTab.partitionColumnNames.length) ++ - Seq(StructField("c3", IntegerType)) ++ - oldTab.schema.takeRight(oldTab.partitionColumnNames.length)) - assert(newTab.schema.toString == oldTabSchema.toString) - } + // construct the expected table schema + val oldTabSchema = StructType(oldTab.dataSchema.fields ++ + Seq(StructField("c3", IntegerType)) ++ oldTab.partitionSchema) + assert(newTab.schema == oldTabSchema) } } 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 05d6af1e4f11..7030e7feff54 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 @@ -190,7 +190,7 @@ case class AlterTableAddColumnsCommand( columns: Seq[StructField]) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - val catalogTable = verifyAlterTableAddColumn(catalog, table) + verifyAlterTableAddColumn(catalog, table) try { sparkSession.catalog.uncacheTable(table.quotedString) @@ -202,32 +202,12 @@ case class AlterTableAddColumnsCommand( // Invalidate the table last, otherwise uncaching the table would load the logical plan // back into the hive metastore cache catalog.refreshTable(table) - val partitionFields = catalogTable.schema.takeRight(catalogTable.partitionColumnNames.length) - val newSchemaFields = catalogTable.schema - .take(catalogTable.schema.length - catalogTable.partitionColumnNames.length) ++ - columns ++ partitionFields - checkDuplication(sparkSession, newSchemaFields) - catalog.alterTableSchema(table, newSchema = - catalogTable.schema.copy(fields = newSchemaFields.toArray)) + catalog.alterTableAddColumns( + table, columns, sparkSession.sessionState.conf.caseSensitiveAnalysis) Seq.empty[Row] } - private def checkDuplication(sparkSession: SparkSession, fields: Seq[StructField]): Unit = { - val columnNames = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { - fields.map(_.name) - } else { - fields.map(_.name.toLowerCase) - } - if (columnNames.distinct.length != columnNames.length) { - val duplicateColumns = columnNames.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => x - } - throw new AnalysisException( - s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}") - } - } - /** * ALTER TABLE ADD COLUMNS command does not support temporary view/table, * view, or datasource table with text, orc formats or external provider. @@ -235,7 +215,7 @@ case class AlterTableAddColumnsCommand( */ private def verifyAlterTableAddColumn( catalog: SessionCatalog, - table: TableIdentifier): CatalogTable = { + table: TableIdentifier): Unit = { val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) if (catalogTable.tableType == CatalogTableType.VIEW) { @@ -263,8 +243,6 @@ case class AlterTableAddColumnsCommand( """.stripMargin) } } - - catalogTable } } From 4391edd998fbdab9692d09bee9395fc44d42a2d7 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Sat, 18 Mar 2017 02:00:39 -0700 Subject: [PATCH 30/34] change SessionCatalog.alterTableAddColumn back to alterTableSchema --- .../sql/catalyst/catalog/SessionCatalog.scala | 49 +++++++++++++------ .../catalog/SessionCatalogSuite.scala | 5 +- .../spark/sql/execution/command/tables.scala | 11 ++--- .../sql/hive/execution/HiveDDLSuite.scala | 29 ++++++++--- 4 files changed, 62 insertions(+), 32 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 a72eb24b76e4..00066ccc5cfd 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 @@ -297,31 +297,51 @@ class SessionCatalog( } /** - * Alter the schema of a table identified by the provided table identifier to add new columns + * Alter the schema of a table identified by the provided table identifier. The new schema + * should still contain the existing bucket columns and partition columns used by the table. This + * method will also update any Spark SQL-related parameters stored as Hive table properties (such + * as the schema itself). + * * @param identifier TableIdentifier - * @param columns new columns - * @param caseSensitive enforce case sensitivity for column names + * @param newSchema Updated schema to be used for the table (must contain existing partition and + * bucket columns) */ - def alterTableAddColumns( + def alterTableSchema( identifier: TableIdentifier, - columns: Seq[StructField], - caseSensitive: Boolean): Unit = { + newSchema: StructType): Unit = { val db = formatDatabaseName(identifier.database.getOrElse(getCurrentDatabase)) val table = formatTableName(identifier.table) val tableIdentifier = TableIdentifier(table, Some(db)) requireDbExists(db) requireTableExists(tableIdentifier) - val catalogTable = externalCatalog.getTable(db, table) + val oldSchema = catalogTable.schema + + // no supporting dropping columns yet + if (!oldSchema.forall(f => columnNameResolved(newSchema, f.name ))) { + throw new AnalysisException( + s""" + |Some existing schema fields are not present in the new schema. + |We don't support dropping columns yet. + """.stripMargin) + } + + checkDuplication(newSchema) + // make sure partition columns are at the end val partitionSchema = catalogTable.partitionSchema - // reorder schema columns w.r.t partition columns - val newSchemaFields = catalogTable.dataSchema.fields ++ columns ++ partitionSchema.fields - checkDuplication(newSchemaFields, caseSensitive) - externalCatalog.alterTableSchema(db, table, catalogTable.schema.copy(fields = newSchemaFields)) + val reorderedSchema = newSchema + .filterNot(f => columnNameResolved(partitionSchema, f.name)) ++ partitionSchema + + externalCatalog.alterTableSchema( + db, table, oldSchema.copy(fields = reorderedSchema.toArray)) } - private def checkDuplication(fields: Seq[StructField], caseSensitive: Boolean): Unit = { - val columnNames = if (caseSensitive) { + private def columnNameResolved(schema: StructType, colName: String): Boolean = { + schema.fields.map(_.name).find(conf.resolver(_, colName)).isDefined + } + + private def checkDuplication(fields: Seq[StructField]): Unit = { + val columnNames = if (conf.caseSensitiveAnalysis) { fields.map(_.name) } else { fields.map(_.name.toLowerCase) @@ -330,8 +350,7 @@ class SessionCatalog( val duplicateColumns = columnNames.groupBy(identity).collect { case (x, ys) if ys.length > 1 => x } - throw new AnalysisException( - s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}") + throw new AnalysisException(s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}") } } 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 fd565d49e39c..27a14c3383d4 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 @@ -455,9 +455,8 @@ abstract class SessionCatalogSuite extends PlanTest { withBasicCatalog { sessionCatalog => sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false) val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1") - sessionCatalog.alterTableAddColumns( - TableIdentifier("t1", Some("default")), - Seq(StructField("c3", IntegerType)), false) + sessionCatalog.alterTableSchema( + TableIdentifier("t1", Some("default")), oldTab.schema.add("c3", IntegerType)) val newTab = sessionCatalog.externalCatalog.getTable("default", "t1") // construct the expected table schema 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 7030e7feff54..892af181e66f 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 @@ -190,7 +190,7 @@ case class AlterTableAddColumnsCommand( columns: Seq[StructField]) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - verifyAlterTableAddColumn(catalog, table) + val catalogTable = verifyAlterTableAddColumn(catalog, table) try { sparkSession.catalog.uncacheTable(table.quotedString) @@ -199,11 +199,9 @@ case class AlterTableAddColumnsCommand( log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e) } - // Invalidate the table last, otherwise uncaching the table would load the logical plan - // back into the hive metastore cache catalog.refreshTable(table) - catalog.alterTableAddColumns( - table, columns, sparkSession.sessionState.conf.caseSensitiveAnalysis) + catalog.alterTableSchema( + table, catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns)) Seq.empty[Row] } @@ -215,7 +213,7 @@ case class AlterTableAddColumnsCommand( */ private def verifyAlterTableAddColumn( catalog: SessionCatalog, - table: TableIdentifier): Unit = { + table: TableIdentifier): CatalogTable = { val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) if (catalogTable.tableType == CatalogTableType.VIEW) { @@ -243,6 +241,7 @@ case class AlterTableAddColumnsCommand( """.stripMargin) } } + catalogTable } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 33c2d5a8bfd6..b8837a4ae962 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1931,18 +1931,31 @@ class HiveDDLSuite } } - test("alter table add columns with existing partition column name") { - withTable("tab") { - sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET") - val e = intercept[AnalysisException] { - sql("ALTER TABLE tab ADD COLUMNS (c2 string)") - }.getMessage - assert(e.contains("Found duplicate column(s)")) + Seq("true", "false").foreach { caseSensitive => + test(s"alter add columns with existing partition column name - caseSensitive $caseSensitive") { + withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { + withTable("tab") { + sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET") + if (caseSensitive == "false") { + val e = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (C2 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } else { + // hive catalog will still complains that c1 is duplicate column name because hive + // identifiers are case insensitive. + val e = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (C2 string)") + }.getMessage + assert(e.contains("HiveException")) + } + } + } } } Seq("true", "false").foreach { caseSensitive => - test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { + test(s"alter add columns with existing column name - caseSensitive $caseSensitive") { withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { withTable("t1") { sql("CREATE TABLE t1 (c1 int) USING PARQUET") From a3fef128e7165a3c750457b1cb7d61f043e9725d Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Sat, 18 Mar 2017 13:41:26 -0700 Subject: [PATCH 31/34] update upon review comments --- .../sql/catalyst/catalog/SessionCatalog.scala | 45 +++++++-------- .../spark/sql/execution/command/tables.scala | 1 - .../sql/execution/command/DDLSuite.scala | 55 +++++++++++-------- .../sql/hive/execution/HiveDDLSuite.scala | 49 +++++++---------- 4 files changed, 75 insertions(+), 75 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 00066ccc5cfd..36c954f39400 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 @@ -162,6 +162,20 @@ class SessionCatalog( throw new TableAlreadyExistsException(db = db, table = name.table) } } + + private def checkDuplication(fields: Seq[StructField]): Unit = { + val columnNames = if (conf.caseSensitiveAnalysis) { + fields.map(_.name) + } else { + fields.map(_.name.toLowerCase) + } + if (columnNames.distinct.length != columnNames.length) { + val duplicateColumns = columnNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => x + } + throw new AnalysisException(s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}") + } + } // ---------------------------------------------------------------------------- // Databases // ---------------------------------------------------------------------------- @@ -314,44 +328,31 @@ class SessionCatalog( val tableIdentifier = TableIdentifier(table, Some(db)) requireDbExists(db) requireTableExists(tableIdentifier) + checkDuplication(newSchema) + val catalogTable = externalCatalog.getTable(db, table) val oldSchema = catalogTable.schema - // no supporting dropping columns yet - if (!oldSchema.forall(f => columnNameResolved(newSchema, f.name ))) { + // not supporting dropping columns yet + val nonExistentColumnNames = oldSchema.map(_.name).filterNot(columnNameResolved(newSchema, _)) + if (nonExistentColumnNames.nonEmpty) { throw new AnalysisException( s""" - |Some existing schema fields are not present in the new schema. - |We don't support dropping columns yet. + |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are + |not present in the new schema. We don't support dropping columns yet. """.stripMargin) } - checkDuplication(newSchema) // make sure partition columns are at the end val partitionSchema = catalogTable.partitionSchema val reorderedSchema = newSchema .filterNot(f => columnNameResolved(partitionSchema, f.name)) ++ partitionSchema - externalCatalog.alterTableSchema( - db, table, oldSchema.copy(fields = reorderedSchema.toArray)) + externalCatalog.alterTableSchema(db, table, StructType(reorderedSchema)) } private def columnNameResolved(schema: StructType, colName: String): Boolean = { - schema.fields.map(_.name).find(conf.resolver(_, colName)).isDefined - } - - private def checkDuplication(fields: Seq[StructField]): Unit = { - val columnNames = if (conf.caseSensitiveAnalysis) { - fields.map(_.name) - } else { - fields.map(_.name.toLowerCase) - } - if (columnNames.distinct.length != columnNames.length) { - val duplicateColumns = columnNames.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => x - } - throw new AnalysisException(s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}") - } + schema.fields.map(_.name).exists(conf.resolver(_, colName)) } /** 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 892af181e66f..35aaca5ad731 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 @@ -198,7 +198,6 @@ case class AlterTableAddColumnsCommand( case NonFatal(e) => log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e) } - catalog.refreshTable(table) catalog.alterTableSchema( table, catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns)) 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 a9366e7ddeb7..336cec32cc56 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 @@ -165,26 +165,6 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)")) } } - - Seq("true", "false").foreach { caseSensitive => - test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { - withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { - withTable("t1") { - sql("CREATE TABLE t1 (c1 int) USING PARQUET") - if (caseSensitive == "false") { - val e = intercept[AnalysisException] { - sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") - }.getMessage - assert(e.contains("Found duplicate column(s)")) - } else { - sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") - assert(sql("SELECT * FROM t1").schema - .equals(new StructType().add("c1", IntegerType).add("C1", StringType))) - } - } - } - } - } } abstract class DDLSuite extends QueryTest with SQLTestUtils { @@ -2205,7 +2185,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql("INSERT INTO t1 VALUES (1)") sql("ALTER TABLE t1 ADD COLUMNS (c2 int)") checkAnswer( - sql("SELECT * FROM t1"), + spark.table("t1"), Seq(Row(1, null)) ) checkAnswer( @@ -2229,7 +2209,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)") sql("ALTER TABLE t1 ADD COLUMNS (c3 int)") checkAnswer( - sql("SELECT * FROM t1"), + spark.table("t1"), Seq(Row(1, null, 2)) ) checkAnswer( @@ -2251,7 +2231,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { test("alter datasource table add columns - text format not supported") { withTable("t1") { - sql(s"CREATE TABLE t1 (c1 int) USING text") + sql("CREATE TABLE t1 (c1 int) USING text") val e = intercept[AnalysisException] { sql("ALTER TABLE t1 ADD COLUMNS (c2 int)") }.getMessage @@ -2300,4 +2280,33 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } } + + Seq("true", "false").foreach { caseSensitive => + test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { + withTable("t1") { + sql("CREATE TABLE t1 (c1 int) USING PARQUET") + if (caseSensitive == "false") { + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } else { + if (isUsingHiveMetastore) { + // hive catalog will still complains that c1 is duplicate column name because hive + // identifiers are case insensitive. + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + }.getMessage + assert(e.contains("HiveException")) + } else { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + assert(spark.table("t1").schema + .equals(new StructType().add("c1", IntegerType).add("C1", StringType))) + } + } + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index b8837a4ae962..9b25a02ea50d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1877,11 +1877,11 @@ class HiveDDLSuite sql("SELECT * FROM tab WHERE c3 = 1"), Seq(Row(1, 2, null, 1)) ) - assert(sql("SELECT * FROM tab").schema + assert(spark.table("tab").schema .contains(StructField("c4", IntegerType))) sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)") checkAnswer( - sql("SELECT * FROM tab"), + spark.table("tab"), Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2)) ) checkAnswer( @@ -1902,7 +1902,7 @@ class HiveDDLSuite sql("SELECT * FROM tab WHERE c4 IS NULL"), Seq(Row(1, 2, null)) ) - assert(sql("SELECT * FROM tab").schema + assert(spark.table("tab").schema .contains(StructField("c4", IntegerType))) sql("INSERT INTO tab VALUES (2, 3, 4)") checkAnswer( @@ -1910,7 +1910,7 @@ class HiveDDLSuite Seq(Row(2, 3, 4)) ) checkAnswer( - sql("SELECT * FROM tab"), + spark.table("tab"), Seq(Row(1, 2, null), Row(2, 3, 4)) ) } @@ -1932,45 +1932,36 @@ class HiveDDLSuite } Seq("true", "false").foreach { caseSensitive => - test(s"alter add columns with existing partition column name - caseSensitive $caseSensitive") { - withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { + test(s"alter add columns with existing column name - caseSensitive $caseSensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { withTable("tab") { sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET") if (caseSensitive == "false") { - val e = intercept[AnalysisException] { + // duplicating partitioning column name + val e1 = intercept[AnalysisException] { sql("ALTER TABLE tab ADD COLUMNS (C2 string)") }.getMessage - assert(e.contains("Found duplicate column(s)")) + assert(e1.contains("Found duplicate column(s)")) + + // duplicating data column name + val e2 = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (C1 string)") + }.getMessage + assert(e2.contains("Found duplicate column(s)")) } else { // hive catalog will still complains that c1 is duplicate column name because hive // identifiers are case insensitive. - val e = intercept[AnalysisException] { + val e1 = intercept[AnalysisException] { sql("ALTER TABLE tab ADD COLUMNS (C2 string)") }.getMessage - assert(e.contains("HiveException")) - } - } - } - } - } + assert(e1.contains("HiveException")) - Seq("true", "false").foreach { caseSensitive => - test(s"alter add columns with existing column name - caseSensitive $caseSensitive") { - withSQLConf(("spark.sql.caseSensitive", caseSensitive)) { - withTable("t1") { - sql("CREATE TABLE t1 (c1 int) USING PARQUET") - if (caseSensitive == "false") { - val e = intercept[AnalysisException] { - sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") - }.getMessage - assert(e.contains("Found duplicate column(s)")) - } else { // hive catalog will still complains that c1 is duplicate column name because hive // identifiers are case insensitive. - val e = intercept[AnalysisException] { - sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + val e2 = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (C1 string)") }.getMessage - assert(e.contains("HiveException")) + assert(e2.contains("HiveException")) } } } From 1eb7cd3ccc6e8ba02510ef5d5cce0beb31ffd8f6 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Sat, 18 Mar 2017 23:15:09 -0700 Subject: [PATCH 32/34] some minor updates upon review comments --- .../catalyst/catalog/SessionCatalogSuite.scala | 16 ++++++++++++++-- .../spark/sql/execution/command/tables.scala | 6 +++--- .../spark/sql/execution/command/DDLSuite.scala | 7 +++++-- .../spark/sql/hive/execution/HiveDDLSuite.scala | 5 +++++ 4 files changed, 27 insertions(+), 7 deletions(-) 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 27a14c3383d4..b13264ed0525 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 @@ -460,9 +460,21 @@ abstract class SessionCatalogSuite extends PlanTest { val newTab = sessionCatalog.externalCatalog.getTable("default", "t1") // construct the expected table schema - val oldTabSchema = StructType(oldTab.dataSchema.fields ++ + val expectedTableSchema = StructType(oldTab.dataSchema.fields ++ Seq(StructField("c3", IntegerType)) ++ oldTab.partitionSchema) - assert(newTab.schema == oldTabSchema) + assert(newTab.schema == expectedTableSchema) + } + } + + test("alter table drop columns") { + withBasicCatalog { sessionCatalog => + sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false) + val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1") + val e = intercept[AnalysisException] { + sessionCatalog.alterTableSchema( + TableIdentifier("t1", Some("default")), StructType(oldTab.schema.drop(1))) + }.getMessage + assert(e.contains("We don't support dropping columns yet.")) } } 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 35aaca5ad731..004d704dbec4 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 @@ -235,9 +235,9 @@ case class AlterTableAddColumnsCommand( case s => throw new AnalysisException( s""" - |ALTER ADD COLUMNS does not support datasource table with type $s. - |You must drop and re-create the table for adding the new columns. Tables: $table - """.stripMargin) + |ALTER ADD COLUMNS does not support datasource table with type $s. + |You must drop and re-create the table for adding the new columns. Tables: $table + """.stripMargin) } } catalogTable 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 336cec32cc56..6e24cf22fe42 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 @@ -165,6 +165,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)")) } } + } abstract class DDLSuite extends QueryTest with SQLTestUtils { @@ -2178,7 +2179,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - Seq("parquet", "json", "csv").foreach { provider => + val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv") + + supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider => test(s"alter datasource table add columns - $provider") { withTable("t1") { sql(s"CREATE TABLE t1 (c1 int) USING $provider") @@ -2202,7 +2205,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - Seq("parquet", "json", "csv").foreach { provider => + supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider => test(s"alter datasource table add columns - partitioned - $provider") { withTable("t1") { sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 9b25a02ea50d..9da2f5251892 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1873,6 +1873,7 @@ class HiveDDLSuite sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)") sql("ALTER TABLE tab ADD COLUMNS (c4 int)") + checkAnswer( sql("SELECT * FROM tab WHERE c3 = 1"), Seq(Row(1, 2, null, 1)) @@ -1888,6 +1889,10 @@ class HiveDDLSuite sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"), Seq(Row(2, 3, 4, 2)) ) + + sql("ALTER TABLE tab ADD COLUMNS (c5 char(10))") + assert(spark.table("tab").schema.find(_.name == "c5") + .get.metadata.getString("HIVE_TYPE_STRING") == "char(10)") } } } From 04ce8f4853d7ffb8d3b33853c88add6d122e9aed Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Mon, 20 Mar 2017 19:47:25 -0700 Subject: [PATCH 33/34] update based on review --- .../sql/catalyst/catalog/SessionCatalog.scala | 12 ++++------- .../catalog/SessionCatalogSuite.scala | 4 +++- .../spark/sql/execution/command/tables.scala | 5 ++++- .../sql/execution/command/DDLSuite.scala | 18 +++-------------- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 20 +------------------ .../spark/sql/sources/TableScanSuite.scala | 18 ----------------- .../sql/hive/execution/HiveDDLSuite.scala | 20 +++---------------- 7 files changed, 18 insertions(+), 79 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 36c954f39400..cf7035f8e76d 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 @@ -318,7 +318,7 @@ class SessionCatalog( * * @param identifier TableIdentifier * @param newSchema Updated schema to be used for the table (must contain existing partition and - * bucket columns) + * bucket columns, and partition columns need to be at the end) */ def alterTableSchema( identifier: TableIdentifier, @@ -343,15 +343,11 @@ class SessionCatalog( """.stripMargin) } - // make sure partition columns are at the end - val partitionSchema = catalogTable.partitionSchema - val reorderedSchema = newSchema - .filterNot(f => columnNameResolved(partitionSchema, f.name)) ++ partitionSchema - - externalCatalog.alterTableSchema(db, table, StructType(reorderedSchema)) + // assuming the newSchema has all partition columns at the end as required + externalCatalog.alterTableSchema(db, table, StructType(newSchema)) } - private def columnNameResolved(schema: StructType, colName: String): Boolean = { + def columnNameResolved(schema: StructType, colName: String): Boolean = { schema.fields.map(_.name).exists(conf.resolver(_, colName)) } 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 b13264ed0525..05759d07c99f 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 @@ -455,8 +455,10 @@ abstract class SessionCatalogSuite extends PlanTest { withBasicCatalog { sessionCatalog => sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false) val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1") + val partitionSchema = oldTab.partitionSchema sessionCatalog.alterTableSchema( - TableIdentifier("t1", Some("default")), oldTab.schema.add("c3", IntegerType)) + TableIdentifier("t1", Some("default")), + StructType(oldTab.dataSchema.add("c3", IntegerType) ++ partitionSchema)) val newTab = sessionCatalog.externalCatalog.getTable("default", "t1") // construct the expected table schema 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 004d704dbec4..93307fc88356 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 @@ -199,8 +199,11 @@ case class AlterTableAddColumnsCommand( log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e) } catalog.refreshTable(table) + + // make sure any partition columns are at the end of the fields + val reorderedSchema = catalogTable.dataSchema ++ columns ++ catalogTable.partitionSchema catalog.alterTableSchema( - table, catalogTable.schema.copy(fields = catalogTable.schema.fields ++ columns)) + table, catalogTable.schema.copy(fields = reorderedSchema.toArray)) Seq.empty[Row] } 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 6e24cf22fe42..9ed5b5f2cb5e 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 @@ -2272,24 +2272,12 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("alter table add columns to table referenced by a view") { - withTable("t1") { - withView("v1") { - sql("CREATE TABLE t1 (c1 int, c2 int) USING PARQUET") - sql("CREATE VIEW v1 AS SELECT * FROM t1") - val originViewSchema = sql("SELECT * FROM v1").schema - sql("ALTER TABLE t1 ADD COLUMNS (c3 int)") - assert(sql("SELECT * FROM v1").schema == originViewSchema) - } - } - } - - Seq("true", "false").foreach { caseSensitive => + Seq(true, false).foreach { caseSensitive => test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { withTable("t1") { sql("CREATE TABLE t1 (c1 int) USING PARQUET") - if (caseSensitive == "false") { + if (!caseSensitive) { val e = intercept[AnalysisException] { sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") }.getMessage diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 09307319b81c..5463728ca0c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -965,27 +965,9 @@ class JDBCSuite extends SparkFunSuite |CREATE TEMPORARY VIEW people_view |USING org.apache.spark.sql.jdbc |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass') - """.stripMargin) + """.stripMargin.replaceAll("\n", " ")) assert(sql("select * from people_view").count() == 3) } } - - test("ALTER TABLE ADD COLUMNS") { - withTable("ds_jdbc") { - sql( - s""" - |CREATE TABLE IF NOT EXISTS ds_jdbc - |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass') - """.stripMargin) - - conn.prepareStatement("ALTER TABLE TEST.PEOPLE ADD COLUMN (C3 int)").executeUpdate() - conn.commit() - val e = intercept[AnalysisException] { - sql("ALTER TABLE ds_jdbc ADD COLUMNS (C3 int)") - }.getMessage - assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type")) - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 2ffd490f93b3..b01d15eb917e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -416,22 +416,4 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { val comments = planned.schema.fields.map(_.getComment().getOrElse("NO_COMMENT")).mkString(",") assert(comments === "SN,SA,NO_COMMENT") } - - test("ALTER TABLE ADD COLUMNS does not support RelationProvider") { - withTable("tab") { - sql( - """ - |CREATE TABLE tab - |USING org.apache.spark.sql.sources.SimpleScanSource - |OPTIONS ( - | From '1', - | To '10' - |) - """.stripMargin) - val e = intercept[AnalysisException] { - sql("ALTER TABLE tab ADD COLUMNS (c3 int)") - }.getMessage - assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type")) - } - } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 9da2f5251892..04bc79d43032 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1922,26 +1922,12 @@ class HiveDDLSuite } } - Seq("orc", "ORC", "org.apache.spark.sql.hive.orc", - "org.apache.spark.sql.hive.orc.DefaultSource").foreach { source => - test(s"alter datasource table add columns - $source format not supported") { - withTable("tab") { - sql(s"CREATE TABLE tab (c1 int) USING $source") - val e = intercept[AnalysisException] { - sql("ALTER TABLE tab ADD COLUMNS (c2 int)") - }.getMessage - assert( - e.contains(s"ALTER ADD COLUMNS does not support datasource table with type")) - } - } - } - - Seq("true", "false").foreach { caseSensitive => + Seq(true, false).foreach { caseSensitive => test(s"alter add columns with existing column name - caseSensitive $caseSensitive") { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { withTable("tab") { sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET") - if (caseSensitive == "false") { + if (!caseSensitive) { // duplicating partitioning column name val e1 = intercept[AnalysisException] { sql("ALTER TABLE tab ADD COLUMNS (C2 string)") From 7d8437dbba76e1e88c9f230dc7ea895d343dceb8 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Mon, 20 Mar 2017 20:49:33 -0700 Subject: [PATCH 34/34] update on minor comments --- .../apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 4 ++-- .../spark/sql/catalyst/catalog/SessionCatalogSuite.scala | 3 +-- 2 files changed, 3 insertions(+), 4 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 cf7035f8e76d..db2f99abce35 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 @@ -344,10 +344,10 @@ class SessionCatalog( } // assuming the newSchema has all partition columns at the end as required - externalCatalog.alterTableSchema(db, table, StructType(newSchema)) + externalCatalog.alterTableSchema(db, table, newSchema) } - def columnNameResolved(schema: StructType, colName: String): Boolean = { + private def columnNameResolved(schema: StructType, colName: String): Boolean = { schema.fields.map(_.name).exists(conf.resolver(_, colName)) } 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 05759d07c99f..c9227c4055a1 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 @@ -455,10 +455,9 @@ abstract class SessionCatalogSuite extends PlanTest { withBasicCatalog { sessionCatalog => sessionCatalog.createTable(newTable("t1", "default"), ignoreIfExists = false) val oldTab = sessionCatalog.externalCatalog.getTable("default", "t1") - val partitionSchema = oldTab.partitionSchema sessionCatalog.alterTableSchema( TableIdentifier("t1", Some("default")), - StructType(oldTab.dataSchema.add("c3", IntegerType) ++ partitionSchema)) + StructType(oldTab.dataSchema.add("c3", IntegerType) ++ oldTab.partitionSchema)) val newTab = sessionCatalog.externalCatalog.getTable("default", "t1") // construct the expected table schema