From 9a2155103c5426dbc9a6cca81b4d6b02e07e7fef Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 3 Jun 2019 17:38:15 -0700 Subject: [PATCH 01/22] Atomic CTAS --- .../catalog/v2/TransactionalTableCatalog.java | 39 ++++++++++ .../spark/sql/sources/v2/StagedTable.java | 25 ++++++ .../v2/WriteToDataSourceV2Exec.scala | 78 ++++++++++++------- 3 files changed, 116 insertions(+), 26 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TransactionalTableCatalog.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TransactionalTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TransactionalTableCatalog.java new file mode 100644 index 000000000000..9f27d74e639a --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TransactionalTableCatalog.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalog.v2; + +import java.util.Map; + +import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.sources.v2.StagedTable; +import org.apache.spark.sql.types.StructType; + +public interface TransactionalTableCatalog { + + StagedTable stageCreate( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties); + + StagedTable stageReplace( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java new file mode 100644 index 000000000000..5c5ff5c12c38 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +public interface StagedTable extends Table { + + void commitStagedChanges(); + + void abortStagedChanges(); +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 6c771ea98832..b7c73b1f531f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -26,7 +26,7 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.executor.CommitDeniedException import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TransactionalTableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException @@ -69,32 +69,58 @@ case class CreateTableAsSelectExec( import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper override protected def doExecute(): RDD[InternalRow] = { - if (catalog.tableExists(ident)) { - if (ifNotExists) { - return sparkContext.parallelize(Seq.empty, 1) - } - - throw new TableAlreadyExistsException(ident) + catalog match { + case txnCatalog: TransactionalTableCatalog => + val stagedTable = txnCatalog.stageCreate( + ident, query.schema, partitioning.toArray, properties.asJava) + Utils.tryWithSafeFinallyAndFailureCallbacks({ + stagedTable match { + case table: SupportsWrite => + val batchWrite = table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + .buildForBatch() + + val writtenRows = doWrite(batchWrite) + + stagedTable.commitStagedChanges() + writtenRows + case _ => + // table does not support writes + throw new SparkException( + s"Table implementation does not support writes: ${ident.quoted}") + } + })(catchBlock = { + stagedTable.abortStagedChanges() + }) + case _ => + Utils.tryWithSafeFinallyAndFailureCallbacks({ + if (catalog.tableExists(ident)) { + if (ifNotExists) { + return sparkContext.parallelize(Seq.empty, 1) + } + + throw new TableAlreadyExistsException(ident) + } + catalog.createTable( + ident, query.schema, partitioning.toArray, properties.asJava) match { + case table: SupportsWrite => + val batchWrite = table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + .buildForBatch() + + doWrite(batchWrite) + + case _ => + // table does not support writes + throw new SparkException( + s"Table implementation does not support writes: ${ident.quoted}") + } + })(catchBlock = { + catalog.dropTable(ident) + }) } - - Utils.tryWithSafeFinallyAndFailureCallbacks({ - catalog.createTable(ident, query.schema, partitioning.toArray, properties.asJava) match { - case table: SupportsWrite => - val batchWrite = table.newWriteBuilder(writeOptions) - .withInputDataSchema(query.schema) - .withQueryId(UUID.randomUUID().toString) - .buildForBatch() - - doWrite(batchWrite) - - case _ => - // table does not support writes - throw new SparkException(s"Table implementation does not support writes: ${ident.quoted}") - } - - })(catchBlock = { - catalog.dropTable(ident) - }) } } From 266784e52b329223252a175d8b4f607b94d06b39 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 4 Jun 2019 14:31:56 -0700 Subject: [PATCH 02/22] Wire together the rest of replace table logical plans. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 12 ++ .../sql/catalyst/parser/AstBuilder.scala | 82 +++++++++++++- .../plans/logical/basicLogicalOperators.scala | 36 ++++++ .../logical/sql/ReplaceTableStatement.scala | 65 +++++++++++ .../datasources/DataSourceResolution.scala | 71 +++++++++++- .../datasources/v2/DataSourceV2Strategy.scala | 15 ++- .../datasources/v2/ReplaceTableExec.scala | 56 +++++++++ .../v2/WriteToDataSourceV2Exec.scala | 107 +++++++++++++++--- 8 files changed, 424 insertions(+), 20 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala 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 91beb5e639af..4c20d2006ab4 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 @@ -111,6 +111,14 @@ statement (AS? query)? #createHiveTable | CREATE TABLE (IF NOT EXISTS)? target=tableIdentifier LIKE source=tableIdentifier locationSpec? #createTableLike + | replaceTableHeader ('(' colTypeList ')')? tableProvider + ((OPTIONS options=tablePropertyList) | + (PARTITIONED BY partitioning=transformList) | + bucketSpec | + locationSpec | + (COMMENT comment=STRING) | + (TBLPROPERTIES tableProps=tablePropertyList))* + (AS? query)? #replaceTable | ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS (identifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze | ALTER TABLE tableIdentifier @@ -244,6 +252,10 @@ createTableHeader : CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? multipartIdentifier ; +replaceTableHeader + : REPLACE TEMPORARY? TABLE multipartIdentifier + ; + bucketSpec : CLUSTERED BY identifierList (SORTED BY orderedIdentifierList)? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index fa05efebf9c6..fcc2b96dce23 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -2035,6 +2035,16 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging (multipartIdentifier, temporary, ifNotExists, ctx.EXTERNAL != null) } + /** + * Validate a replace table statement and return the [[TableIdentifier]]. + */ + override def visitReplaceTableHeader( + ctx: ReplaceTableHeaderContext): TableHeader = withOrigin(ctx) { + val temporary = ctx.TEMPORARY != null + val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText) + (multipartIdentifier, temporary, false, false) + } + /** * Parse a list of transforms. */ @@ -2195,6 +2205,76 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Replace a table, returning a [[ReplaceTableStatement]] logical plan. + * + * Expected format: + * {{{ + * REPLACE TABLE [IF NOT EXISTS] [db_name.]table_name + * USING table_provider + * replace_table_clauses + * [[AS] select_statement]; + * + * replace_table_clauses (order insensitive): + * [OPTIONS table_property_list] + * [PARTITIONED BY (col_name, transform(col_name), transform(constant, col_name), ...)] + * [CLUSTERED BY (col_name, col_name, ...) + * [SORTED BY (col_name [ASC|DESC], ...)] + * INTO num_buckets BUCKETS + * ] + * [LOCATION path] + * [COMMENT table_comment] + * [TBLPROPERTIES (property_name=property_value, ...)] + * }}} + */ + override def visitReplaceTable(ctx: ReplaceTableContext): LogicalPlan = withOrigin(ctx) { + val (table, temp, ifNotExists, external) = visitReplaceTableHeader(ctx.replaceTableHeader) + if (external) { + operationNotAllowed("REPLACE EXTERNAL TABLE ... USING", ctx) + } + + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) + checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) + checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + + val schema = Option(ctx.colTypeList()).map(createSchema) + val partitioning: Seq[Transform] = + Option(ctx.partitioning).map(visitTransformList).getOrElse(Nil) + val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) + val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + + val provider = ctx.tableProvider.qualifiedName.getText + val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) + val comment = Option(ctx.comment).map(string) + + Option(ctx.query).map(plan) match { + case Some(_) if temp => + operationNotAllowed("REPLACE TEMPORARY TABLE ... USING ... AS query", ctx) + + case Some(_) if schema.isDefined => + operationNotAllowed( + "Schema may not be specified in a Replace Table As Select (RTAS) statement", + ctx) + + case Some(query) => + ReplaceTableAsSelectStatement( + table, query, partitioning, bucketSpec, properties, provider, options, location, comment) + + case None if temp => + // REPLACE TEMPORARY TABLE ... USING ... is not supported by the catalyst parser. + // Use REPLACE TEMPORARY VIEW ... USING ... instead. + operationNotAllowed("REPLACE TEMPORARY TABLE IF NOT EXISTS", ctx) + + case _ => + ReplaceTableStatement(table, schema.getOrElse(new StructType), partitioning, bucketSpec, + properties, provider, options, location, comment) + } + } + /** * Create a [[DropTableStatement]] command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 6bf12cff28f9..e0136508183f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -440,6 +440,42 @@ case class CreateTableAsSelect( } } +/** + * Replace a table with a v2 catalog. + * + * If the table does not exist, it will be created. The persisted table will have no contents + * as a result of this operation. + */ +case class ReplaceTable( + catalog: TableCatalog, + tableName: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + properties: Map[String, String]) extends Command + +/** + * Replaces a table from a select query with a v2 catalog. + * + * If the table does not already exist, it will be created. + */ +case class ReplaceTableAsSelect( + catalog: TableCatalog, + tableName: Identifier, + partitioning: Seq[Transform], + query: LogicalPlan, + properties: Map[String, String], + writeOptions: Map[String, String]) extends Command { + + override def children: Seq[LogicalPlan] = Seq(query) + + override lazy val resolved: Boolean = { + // the table schema is created from the query schema, so the only resolution needed is to check + // that the columns referenced by the table's partitioning exist in the query schema + val references = partitioning.flatMap(_.references).toSet + references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) + } +} + /** * Append data to an existing table. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala new file mode 100644 index 000000000000..05ed4a625297 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical.sql + +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types.StructType + +/** + * A REPLACE TABLE command, as parsed from SQL. + * + * If the table exists prior to running this command, executing this statement + * will replace the table's metadata and clear the underlying rows from the table. + */ +case class ReplaceTableStatement( + tableName: Seq[String], + tableSchema: StructType, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String]) extends ParsedStatement { + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.empty +} + +/** + * A REPLACE TABLE AS SELECT command, as parsed from SQL. + */ +case class ReplaceTableAsSelectStatement( + tableName: Seq[String], + asSelect: LogicalPlan, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String]) extends ParsedStatement { + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq(asSelect) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index 19881f69f158..57528f88686a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -27,8 +27,8 @@ import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} -import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement} +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect} +import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.DropTableCommand import org.apache.spark.sql.internal.SQLConf @@ -85,6 +85,38 @@ case class DataSourceResolution( .asTableCatalog convertCTAS(catalog, identifier, create) + case ReplaceTableStatement( + AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, + V1WriteProvider(provider), options, location, comment) => + throw new AnalysisException( + s"Replacing tables is not supported using the legacy / v1 Spark external catalog" + + s" API. Write provider name: $provider, identifier: $table.") + + case ReplaceTableAsSelectStatement( + AsTableIdentifier(table), query, partitionCols, bucketSpec, properties, + V1WriteProvider(provider), options, location, comment) => + throw new AnalysisException( + s"Replacing tables is not supported using the legacy / v1 Spark external catalog" + + s" API. Write provider name: $provider, identifier: $table.") + + case replace: ReplaceTableStatement => + // the provider was not a v1 source, convert to a v2 plan + val CatalogObjectIdentifier(maybeCatalog, identifier) = replace.tableName + val catalog = maybeCatalog.orElse(defaultCatalog) + .getOrElse(throw new AnalysisException( + s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) + .asTableCatalog + convertReplaceTable(catalog, identifier, replace) + + case rtas: ReplaceTableAsSelectStatement => + // the provider was not a v1 source, convert to a v2 plan + val CatalogObjectIdentifier(maybeCatalog, identifier) = rtas.tableName + val catalog = maybeCatalog.orElse(defaultCatalog) + .getOrElse(throw new AnalysisException( + s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) + .asTableCatalog + convertRTAS(catalog, identifier, rtas) + case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => DropTable(catalog.asTableCatalog, ident, ifExists) @@ -194,6 +226,41 @@ case class DataSourceResolution( ignoreIfExists = create.ifNotExists) } + private def convertRTAS( + catalog: TableCatalog, + identifier: Identifier, + rtas: ReplaceTableAsSelectStatement): ReplaceTableAsSelect = { + // convert the bucket spec and add it as a transform + val partitioning = rtas.partitioning ++ rtas.bucketSpec.map(_.asTransform) + val properties = convertTableProperties( + rtas.properties, rtas.options, rtas.location, rtas.comment, rtas.provider) + + ReplaceTableAsSelect( + catalog, + identifier, + partitioning, + rtas.asSelect, + properties, + writeOptions = rtas.options.filterKeys(_ != "path")) + } + + private def convertReplaceTable( + catalog: TableCatalog, + identifier: Identifier, + replace: ReplaceTableStatement): ReplaceTable = { + // convert the bucket spec and add it as a transform + val partitioning = replace.partitioning ++ replace.bucketSpec.map(_.asTransform) + val properties = convertTableProperties( + replace.properties, replace.options, replace.location, replace.comment, replace.provider) + + ReplaceTable( + catalog, + identifier, + replace.tableSchema, + partitioning, + properties) + } + private def convertTableProperties( properties: Map[String, String], options: Map[String, String], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 27d87960edb3..57f733e982da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} @@ -168,6 +168,19 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { CreateTableAsSelectExec( catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + case ReplaceTable(catalog, ident, schema, parts, props) => + ReplaceTableExec(catalog, ident, schema, parts, props) :: Nil + + case ReplaceTableAsSelect(catalog, ident, parts, query, props, options) => + val writeOptions = new CaseInsensitiveStringMap(options.asJava) + ReplaceTableAsSelectExec( + catalog, ident, parts, planLater(query), props, writeOptions) :: Nil + + case CreateTableAsSelect(catalog, ident, parts, query, props, options, ifNotExists) => + val writeOptions = new CaseInsensitiveStringMap(options.asJava) + CreateTableAsSelectExec( + catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + case AppendData(r: DataSourceV2Relation, query, _) => AppendDataExec(r.table.asWritable, r.options, planLater(query)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala new file mode 100644 index 000000000000..2a348f160e95 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import scala.collection.JavaConverters._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TransactionalTableCatalog} +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.types.StructType + +case class ReplaceTableExec( + catalog: TableCatalog, + identifier: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + tableProperties: Map[String, String]) extends LeafExecNode { + + override protected def doExecute(): RDD[InternalRow] = { + catalog match { + case transactional: TransactionalTableCatalog => + transactional.stageReplace( + identifier, + tableSchema, + partitioning.toArray, + tableProperties.asJava).commitStagedChanges() + case _ => + if (catalog.tableExists(identifier)) { + catalog.dropTable(identifier) + } + catalog.createTable(identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + } + + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index b7c73b1f531f..c38ab5821da8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.sources.{AlwaysTrue, Filter} -import org.apache.spark.sql.sources.v2.SupportsWrite +import org.apache.spark.sql.sources.v2.{StagedTable, SupportsWrite} import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{LongAccumulator, Utils} @@ -54,8 +54,11 @@ case class WriteToDataSourceV2(batchWrite: BatchWrite, query: LogicalPlan) * Physical plan node for v2 create table as select. * * A new table will be created using the schema of the query, and rows from the query are appended. - * If either table creation or the append fails, the table will be deleted. This implementation does - * not provide an atomic CTAS. + * If either table creation or the append fails, the table will be deleted. The CTAS operation is + * atomic if the catalog supports atomic operations, where the creation of the table is staged and + * the commit of the write should bundle the commitment of the metadata and the table contents in + * a single unit. Otherwise, the table is created, and if the write fails, the table is + * subsequently dropped. */ case class CreateTableAsSelectExec( catalog: TableCatalog, @@ -64,44 +67,87 @@ case class CreateTableAsSelectExec( query: SparkPlan, properties: Map[String, String], writeOptions: CaseInsensitiveStringMap, - ifNotExists: Boolean) extends V2TableWriteExec { + ifNotExists: Boolean) extends StagedTableWriteExec { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper override protected def doExecute(): RDD[InternalRow] = { + if (catalog.tableExists(ident)) { + if (ifNotExists) { + return sparkContext.parallelize(Seq.empty, 1) + } + + throw new TableAlreadyExistsException(ident) + } catalog match { case txnCatalog: TransactionalTableCatalog => val stagedTable = txnCatalog.stageCreate( ident, query.schema, partitioning.toArray, properties.asJava) + writeToStagedTable(stagedTable, writeOptions, ident) + case _ => Utils.tryWithSafeFinallyAndFailureCallbacks({ - stagedTable match { + catalog.createTable( + ident, query.schema, partitioning.toArray, properties.asJava) match { case table: SupportsWrite => val batchWrite = table.newWriteBuilder(writeOptions) .withInputDataSchema(query.schema) .withQueryId(UUID.randomUUID().toString) .buildForBatch() - val writtenRows = doWrite(batchWrite) + doWrite(batchWrite) - stagedTable.commitStagedChanges() - writtenRows case _ => // table does not support writes throw new SparkException( s"Table implementation does not support writes: ${ident.quoted}") } })(catchBlock = { - stagedTable.abortStagedChanges() + catalog.dropTable(ident) }) + } + } +} + +/** + * Physical plan node for v2 replace table as select. + * + * A new table will be created using the schema of the query, and rows from the query are appended. + * If the table exists, its contents and schema should be replaced with the schema and the contents + * of the query. If the catalog supports atomic replacement of tables, the plugin's transactional + * implementation is used so that the plugin can avoid losing previously committed data if any + * part of the write fails. The non-atomic implementation will drop the table and then run + * non-atomic CTAS. + */ +case class ReplaceTableAsSelectExec( + catalog: TableCatalog, + ident: Identifier, + partitioning: Seq[Transform], + query: SparkPlan, + properties: Map[String, String], + writeOptions: CaseInsensitiveStringMap) extends StagedTableWriteExec { + + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + + override protected def doExecute(): RDD[InternalRow] = { + catalog match { + case txnCatalog: TransactionalTableCatalog => + val stagedTable = txnCatalog.stageReplace( + ident, query.schema, partitioning.toArray, properties.asJava) + writeToStagedTable(stagedTable, writeOptions, ident) case _ => - Utils.tryWithSafeFinallyAndFailureCallbacks({ - if (catalog.tableExists(ident)) { - if (ifNotExists) { - return sparkContext.parallelize(Seq.empty, 1) - } + // Note that this operation is potentially unsafe, but these are the strict semantics of + // RTAS if the catalog does not support atomic operations. + // + // There are numerous cases we concede to where the table will be dropped and irrecoverable: + // + // 1. Creating the new table fails, + // 2. Writing to the new table fails, + // 3. The table returned by catalog.createTable doesn't support writing. + if (catalog.tableExists(ident)) { + catalog.dropTable(ident) + } - throw new TableAlreadyExistsException(ident) - } + Utils.tryWithSafeFinallyAndFailureCallbacks({ catalog.createTable( ident, query.schema, partitioning.toArray, properties.asJava) match { case table: SupportsWrite => @@ -356,6 +402,35 @@ object DataWritingSparkTask extends Logging { } } +trait StagedTableWriteExec extends V2TableWriteExec { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + + protected def writeToStagedTable( + stagedTable: StagedTable, + writeOptions: CaseInsensitiveStringMap, + ident: Identifier): RDD[InternalRow] = { + Utils.tryWithSafeFinallyAndFailureCallbacks({ + stagedTable match { + case table: SupportsWrite => + val batchWrite = table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + .buildForBatch() + + val writtenRows = doWrite(batchWrite) + stagedTable.commitStagedChanges() + writtenRows + case _ => + // table does not support writes + throw new SparkException( + s"Table implementation does not support writes: ${ident.quoted}") + } + })(catchBlock = { + stagedTable.abortStagedChanges() + }) + } +} + private[v2] case class DataWritingSparkTaskResult( numRows: Long, writerCommitMessage: WriterCommitMessage) From baeabc8ce3d4067d2bdc7f8fc850da32ab032b09 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 4 Jun 2019 16:24:54 -0700 Subject: [PATCH 03/22] Remove redundant code --- .../sql/execution/datasources/v2/DataSourceV2Strategy.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 57f733e982da..56ca182fd989 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -176,11 +176,6 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { ReplaceTableAsSelectExec( catalog, ident, parts, planLater(query), props, writeOptions) :: Nil - case CreateTableAsSelect(catalog, ident, parts, query, props, options, ifNotExists) => - val writeOptions = new CaseInsensitiveStringMap(options.asJava) - CreateTableAsSelectExec( - catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil - case AppendData(r: DataSourceV2Relation, query, _) => AppendDataExec(r.table.asWritable, r.options, planLater(query)) :: Nil From bc8d3b568988c904dff4f31a19f6dc0aa33f4a8b Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 4 Jun 2019 18:43:57 -0700 Subject: [PATCH 04/22] Some unit tests --- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 82 +++++++++++++ .../sources/v2/TestInMemoryTableCatalog.scala | 108 +++++++++++++++++- 2 files changed, 184 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index 5b9071b59b9b..63c3cdd076a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -36,6 +36,8 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn before { spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) + spark.conf.set( + "spark.sql.catalog.testcatatomic", classOf[TestTransactionalInMemoryCatalog].getName) spark.conf.set("spark.sql.default.catalog", "testcat") val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") @@ -46,6 +48,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn after { spark.catalog("testcat").asInstanceOf[TestInMemoryTableCatalog].clearTables() + spark.catalog("testcatatomic").asInstanceOf[TestInMemoryTableCatalog].clearTables() spark.sql("DROP TABLE source") } @@ -170,6 +173,85 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) } + test("ReplaceTableAsSelect: basic v2 implementation using atomic catalog.") { + spark.sql("CREATE TABLE testcatatomic.table_name USING foo AS SELECT id, data FROM source") + val testCatalog = spark.catalog("testcatatomic").asTableCatalog + val originalTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + spark.sql("REPLACE TABLE testcatatomic.table_name USING foo AS SELECT id FROM source") + val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(replacedTable != originalTable, "Table should have been replaced.") + assert(replacedTable.name == "testcatatomic.table_name") + assert(replacedTable.partitioning.isEmpty) + assert(replacedTable.properties == Map("provider" -> "foo").asJava) + assert(replacedTable.schema == new StructType() + .add("id", LongType, nullable = false)) + val rdd = spark.sparkContext.parallelize(replacedTable.asInstanceOf[InMemoryTable].rows) + checkAnswer( + spark.internalCreateDataFrame(rdd, replacedTable.schema), + spark.table("source").select("id")) + } + + test("ReplaceTableAsSelect: Non-atomic catalog creates the empty table, but leaves the" + + " table empty if the write fails.") { + spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) + intercept[Exception] { + spark.sql("REPLACE TABLE testcat.table_name" + + s" USING foo OPTIONS (`${TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION}`=true)" + + s" AS SELECT id FROM source") + } + val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(replacedTable != table, "Table should have been replaced.") + assert(replacedTable.asInstanceOf[InMemoryTable].rows.isEmpty, + "Rows should not have been successfully written to the replaced table.") + } + + test("ReplaceTableAsSelect: Non-atomic catalog drops the table permanently if the" + + " subsequent table creation fails.") { + spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) + intercept[Exception] { + spark.sql("REPLACE TABLE testcat.table_name" + + s" USING foo" + + s" TBLPROPERTIES (`${TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY}`=true)" + + s" AS SELECT id FROM source") + } + assert(!testCatalog.tableExists(Identifier.of(Array(), "table_name")), + "Table should have been dropped and failed to be created.") + } + + test("ReplaceTableAsSelect: Atomic catalog does not drop the table when replace fails.") { + spark.sql("CREATE TABLE testcatatomic.table_name USING foo AS SELECT id, data FROM source") + val testCatalog = spark.catalog("testcatatomic").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "testcatatomic.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + intercept[Exception] { + spark.sql("REPLACE TABLE testcatatomic.table_name" + + s" USING foo OPTIONS (`${TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION}=true)" + + s" AS SELECT id FROM source") + } + var maybeReplacedTable: Table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(maybeReplacedTable === table, "Table should not have changed.") + intercept[Exception] { + spark.sql("REPLACE TABLE testcatatomic.table_name" + + s" USING foo" + + s" TBLPROPERTIES (`${TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY}`=true)" + + s" AS SELECT id FROM source") + } + maybeReplacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(maybeReplacedTable === table, "Table should not have changed.") + } + test("CreateTableAsSelect: use v2 plan because provider is v2") { spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 2ecf1c2f184f..1208761d690c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -23,7 +23,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, TableCatalog, TableChange, TestTableCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, TableCatalog, TableChange, TestTableCatalog, TransactionalTableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} @@ -37,7 +37,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class TestInMemoryTableCatalog extends TableCatalog { import CatalogV2Implicits._ - private val tables: util.Map[Identifier, InMemoryTable] = + protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() private var _name: Option[String] = None @@ -65,11 +65,10 @@ class TestInMemoryTableCatalog extends TableCatalog { schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): Table = { - if (tables.containsKey(ident)) { throw new TableAlreadyExistsException(ident) } - + TestInMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) if (partitions.nonEmpty) { throw new UnsupportedOperationException( s"Catalog $name: Partitioned tables are not supported") @@ -107,7 +106,7 @@ class TestInMemoryTableCatalog extends TableCatalog { /** * A simple in-memory table. Rows are stored as a buffered group produced by each output task. */ -private class InMemoryTable( +class InMemoryTable( val name: String, val schema: StructType, override val properties: util.Map[String, String]) @@ -148,6 +147,7 @@ private class InMemoryTable( } override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + TestInMemoryTableCatalog.maybeSimulateFailedTableWrite(options) new WriteBuilder with SupportsTruncate { private var shouldTruncate: Boolean = false @@ -189,7 +189,103 @@ private class InMemoryTable( } } -private class BufferedRows extends WriterCommitMessage with InputPartition with Serializable { +object TestInMemoryTableCatalog { + val SIMULATE_FAILED_WRITE_OPTION = "spark.sql.test.simulateFailedWrite" + val SIMULATE_FAILED_CREATE_PROPERTY = "spark.sql.test.simulateFailedCreate" + + def maybeSimulateFailedTableCreation(tableProperties: util.Map[String, String]): Unit = { + if (tableProperties.containsKey(TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY) + && tableProperties.get(TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY) + .equalsIgnoreCase("true")) { + throw new IllegalStateException("Manual create table failure.") + } + } + + def maybeSimulateFailedTableWrite(tableOptions: CaseInsensitiveStringMap): Unit = { + if (tableOptions.containsKey(TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION) + && tableOptions.get(TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION) + .equalsIgnoreCase("true")) { + throw new IllegalStateException("Manual write to table failure.") + } + } +} + +class TestTransactionalInMemoryCatalog + extends TestInMemoryTableCatalog with TransactionalTableCatalog { + + override def stageCreate( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + newStagedTable(ident, schema, partitions, properties, false) + } + + override def stageReplace( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + newStagedTable(ident, schema, partitions, properties, true) + } + + private def newStagedTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String], + replaceIfExists: Boolean): StagedTable = { + import CatalogV2Implicits.IdentifierHelper + if (partitions.nonEmpty) { + throw new UnsupportedOperationException( + s"Catalog $name: Partitioned tables are not supported") + } + + TestInMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) + + new TestStagedTable( + ident, + new InMemoryTable(s"$name.${ident.quoted}", schema, properties), + replaceIfExists) + } + + private class TestStagedTable( + ident: Identifier, + delegateTable: InMemoryTable, + replaceIfExists: Boolean) + extends StagedTable with SupportsWrite with SupportsRead { + + override def commitStagedChanges(): Unit = { + if (replaceIfExists) { + tables.put(ident, delegateTable) + } else { + val maybePreCommittedTable = tables.putIfAbsent(ident, delegateTable) + if (maybePreCommittedTable != null) { + throw new TableAlreadyExistsException( + s"Table with identifier $ident and name $name was already created.") + } + } + } + + override def abortStagedChanges(): Unit = {} + + override def name(): String = delegateTable.name + + override def schema(): StructType = delegateTable.schema + + override def capabilities(): util.Set[TableCapability] = delegateTable.capabilities + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + delegateTable.newWriteBuilder(options) + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + delegateTable.newScanBuilder(options) + } + } +} + +class BufferedRows extends WriterCommitMessage with InputPartition with Serializable { val rows = new mutable.ArrayBuffer[InternalRow]() } From 6c958b9368aa7b9b9c12c026627797deca82c0d1 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 5 Jun 2019 13:34:11 -0700 Subject: [PATCH 05/22] DDL parser tests for replace table --- .../sql/catalyst/parser/DDLParserSuite.scala | 543 ++++++++++-------- 1 file changed, 316 insertions(+), 227 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 06cdd067dc28..267d219f7043 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.catalyst.parser -import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} +import java.util.Locale + +import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.types.{IntegerType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -35,82 +37,71 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsePlan(sql), expected, checkAnalysis = false) } - test("create table using - schema") { - val sql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType() - .add("a", IntegerType, nullable = true, "test") - .add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table using - schema") { + val createSql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" + val replaceSql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet", "no viable alternative at input") } - test("create table - with IF NOT EXISTS") { + test("create/replace table - with IF NOT EXISTS") { val sql = "CREATE TABLE IF NOT EXISTS my_tab(a INT, b STRING) USING parquet" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") - } + testCreateOrReplaceDdl( + sql, + TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None), + expectedIfNotExists = true) } - test("create table - with partitioned by") { - val query = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + + test("create/replace table - with partitioned by") { + val createSql = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + "USING parquet PARTITIONED BY (a)" - - parsePlan(query) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType() - .add("a", IntegerType, nullable = true, "test") - .add("b", StringType)) - assert(create.partitioning == Seq(IdentityTransform(FieldReference("a")))) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") + val replaceSql = "REPLACE TABLE my_tab(a INT comment 'test', b STRING) " + + "USING parquet PARTITIONED BY (a)" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType)), + Seq(IdentityTransform(FieldReference("a"))), + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - partitioned by transforms") { - val sql = + test("create/replace table - partitioned by transforms") { + val createSql = """ |CREATE TABLE my_tab (a INT, b STRING, ts TIMESTAMP) USING parquet |PARTITIONED BY ( @@ -123,154 +114,151 @@ class DDLParserSuite extends AnalysisTest { | foo(a, "bar", 34)) """.stripMargin - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType() - .add("a", IntegerType) - .add("b", StringType) - .add("ts", TimestampType)) - assert(create.partitioning == Seq( - IdentityTransform(FieldReference("a")), - BucketTransform(LiteralValue(16, IntegerType), Seq(FieldReference("b"))), - YearsTransform(FieldReference("ts")), - MonthsTransform(FieldReference("ts")), - DaysTransform(FieldReference("ts")), - HoursTransform(FieldReference("ts")), - ApplyTransform("foo", Seq( - FieldReference("a"), - LiteralValue(UTF8String.fromString("bar"), StringType), - LiteralValue(34, IntegerType))))) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + val replaceSql = + """ + |REPLACE TABLE my_tab (a INT, b STRING, ts TIMESTAMP) USING parquet + |PARTITIONED BY ( + | a, + | bucket(16, b), + | years(ts), + | months(ts), + | days(ts), + | hours(ts), + | foo(a, "bar", 34)) + """.stripMargin + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType() + .add("a", IntegerType) + .add("b", StringType) + .add("ts", TimestampType)), + Seq( + IdentityTransform(FieldReference("a")), + BucketTransform(LiteralValue(16, IntegerType), Seq(FieldReference("b"))), + YearsTransform(FieldReference("ts")), + MonthsTransform(FieldReference("ts")), + DaysTransform(FieldReference("ts")), + HoursTransform(FieldReference("ts")), + ApplyTransform("foo", Seq( + FieldReference("a"), + LiteralValue(UTF8String.fromString("bar"), StringType), + LiteralValue(34, IntegerType)))), + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - with bucket") { - val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + + test("create/replace table - with bucket") { + val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" - parsePlan(query) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.contains(BucketSpec(5, Seq("a"), Seq("b")))) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") + val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" + + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + Some(BucketSpec(5, Seq("a"), Seq("b"))), + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - with comment") { - val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.contains("abc")) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table - with comment") { + val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" + val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + Some("abc")) + Seq(createSql, replaceSql).foreach{ sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - with table properties") { - val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet TBLPROPERTIES('test' = 'test')" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties == Map("test" -> "test")) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table - with table properties") { + val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet" + + " TBLPROPERTIES('test' = 'test')" + val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet" + + " TBLPROPERTIES('test' = 'test')" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + None, + Map("test" -> "test"), + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - with location") { - val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.contains("/tmp/file")) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table - with location") { + val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" + val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + Some("/tmp/file"), + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - byte length literal table name") { - val sql = "CREATE TABLE 1m.2g(a INT) USING parquet" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("1m", "2g")) - assert(create.tableSchema == new StructType().add("a", IntegerType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table - byte length literal table name") { + val createSql = "CREATE TABLE 1m.2g(a INT) USING parquet" + val replaceSql = "REPLACE TABLE 1m.2g(a INT) USING parquet" + val expectedTableSpec = TableSpec( + Seq("1m", "2g"), + Some(new StructType().add("a", IntegerType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("Duplicate clauses - create table") { + test("Duplicate clauses - create/replace table") { def createTableHeader(duplicateClause: String): String = { s"CREATE TABLE my_tab(a INT, b STRING) USING parquet $duplicateClause $duplicateClause" } + def replaceTableHeader(duplicateClause: String): String = { + s"CREATE TABLE my_tab(a INT, b STRING) USING parquet $duplicateClause $duplicateClause" + } + intercept(createTableHeader("TBLPROPERTIES('test' = 'test2')"), "Found duplicate clauses: TBLPROPERTIES") intercept(createTableHeader("LOCATION '/tmp/file'"), @@ -281,31 +269,44 @@ class DDLParserSuite extends AnalysisTest { "Found duplicate clauses: CLUSTERED BY") intercept(createTableHeader("PARTITIONED BY (b)"), "Found duplicate clauses: PARTITIONED BY") + + intercept(replaceTableHeader("TBLPROPERTIES('test' = 'test2')"), + "Found duplicate clauses: TBLPROPERTIES") + intercept(replaceTableHeader("LOCATION '/tmp/file'"), + "Found duplicate clauses: LOCATION") + intercept(replaceTableHeader("COMMENT 'a table'"), + "Found duplicate clauses: COMMENT") + intercept(replaceTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS"), + "Found duplicate clauses: CLUSTERED BY") + intercept(replaceTableHeader("PARTITIONED BY (b)"), + "Found duplicate clauses: PARTITIONED BY") } test("support for other types in OPTIONS") { - val sql = + val createSql = """ |CREATE TABLE table_name USING json |OPTIONS (a 1, b 0.1, c TRUE) """.stripMargin - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("table_name")) - assert(create.tableSchema == new StructType) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "json") - assert(create.options == Map("a" -> "1", "b" -> "0.1", "c" -> "true")) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + val replaceSql = + """ + |REPLACE TABLE table_name USING json + |OPTIONS (a 1, b 0.1, c TRUE) + """.stripMargin + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl( + sql, + TableSpec( + Seq("table_name"), + Some(new StructType), + Seq.empty[Transform], + Option.empty[BucketSpec], + Map.empty[String, String], + "json", + Map("a" -> "1", "b" -> "0.1", "c" -> "true"), + None, + None), + expectedIfNotExists = false) } } @@ -340,27 +341,28 @@ class DDLParserSuite extends AnalysisTest { |AS SELECT * FROM src """.stripMargin - checkParsing(s1) - checkParsing(s2) - checkParsing(s3) - - def checkParsing(sql: String): Unit = { - parsePlan(sql) match { - case create: CreateTableAsSelectStatement => - assert(create.tableName == Seq("mydb", "page_view")) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties == Map("p1" -> "v1", "p2" -> "v2")) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.contains("/user/external/page_view")) - assert(create.comment.contains("This is the staging page view table")) - assert(create.ifNotExists) + val s4 = + """ + |REPLACE TABLE mydb.page_view + |USING parquet + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin - case other => - fail(s"Expected to parse ${classOf[CreateTableAsSelectStatement].getClass.getName} " + - s"from query, got ${other.getClass.getName}: $sql") - } + val expectedTableSpec = TableSpec( + Seq("mydb", "page_view"), + None, + Seq.empty[Transform], + None, + Map("p1" -> "v1", "p2" -> "v2"), + "parquet", + Map.empty[String, String], + Some("/user/external/page_view"), + Some("This is the staging page view table")) + Seq(s1, s2, s3, s4).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = true) } } @@ -390,4 +392,91 @@ class DDLParserSuite extends AnalysisTest { parseCompare(s"DROP VIEW view", DropViewStatement(Seq("view"), ifExists = false)) parseCompare(s"DROP VIEW IF EXISTS view", DropViewStatement(Seq("view"), ifExists = true)) } + + private def testCreateOrReplaceDdl( + sqlStatement: String, + tableSpec: TableSpec, + expectedIfNotExists: Boolean) { + val parsedPlan = parsePlan(sqlStatement) + val newTableToken = sqlStatement.split(" ")(0).trim.toUpperCase(Locale.ROOT) + parsedPlan match { + case create: CreateTableStatement if newTableToken == "CREATE" => + assert(create.ifNotExists == expectedIfNotExists) + case ctas: CreateTableAsSelectStatement if newTableToken == "CREATE" => + assert(ctas.ifNotExists == expectedIfNotExists) + case replace: ReplaceTableStatement if newTableToken == "REPLACE" => + case replace: ReplaceTableAsSelectStatement if newTableToken == "REPLACE" => + case other => + fail("First token in statement does not match the expected parsed plan; CREATE TABLE" + + " should create a CreateTableStatement, and REPLACE TABLE should create a" + + s" ReplaceTableStatement. Statement: $sqlStatement, plan type:" + + s" ${parsedPlan.getClass.getName}.") + } + assert(TableSpec(parsedPlan) === tableSpec) + } + + private case class TableSpec( + name: Seq[String], + schema: Option[StructType], + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String]) + + private object TableSpec { + def apply(plan: LogicalPlan): TableSpec = { + plan match { + case create: CreateTableStatement => + TableSpec( + create.tableName, + Some(create.tableSchema), + create.partitioning, + create.bucketSpec, + create.properties, + create.provider, + create.options, + create.location, + create.comment) + case replace: ReplaceTableStatement => + TableSpec( + replace.tableName, + Some(replace.tableSchema), + replace.partitioning, + replace.bucketSpec, + replace.properties, + replace.provider, + replace.options, + replace.location, + replace.comment) + case ctas: CreateTableAsSelectStatement => + TableSpec( + ctas.tableName, + Some(ctas.asSelect).filter(_.resolved).map(_.schema), + ctas.partitioning, + ctas.bucketSpec, + ctas.properties, + ctas.provider, + ctas.options, + ctas.location, + ctas.comment) + case rtas: ReplaceTableAsSelectStatement => + TableSpec( + rtas.tableName, + Some(rtas.asSelect).filter(_.resolved).map(_.schema), + rtas.partitioning, + rtas.bucketSpec, + rtas.properties, + rtas.provider, + rtas.options, + rtas.location, + rtas.comment) + case other => + fail(s"Expected to parse Create, CTAS, Replace, or RTAS plan" + + s" from query, got ${other.getClass.getName}.") + } + } + } } From 08f115e5224ea3d92aa9995988ec3f07967a1cd0 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 7 Jun 2019 14:27:52 -0700 Subject: [PATCH 06/22] Fix merge conflicts --- .../spark/sql/sources/v2/TestInMemoryTableCatalog.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 0f71a8412572..2185691f62f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -23,13 +23,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import scala.collection.mutable -<<<<<<< HEAD import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, TableCatalog, TableChange, TestTableCatalog, TransactionalTableCatalog} -||||||| merged common ancestors -import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, TableCatalog, TableChange, TestTableCatalog} -======= -import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, TableCatalog, TableChange} ->>>>>>> origin/master import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.InternalRow From 89aea5ea3b2bfc6914ada5adf8a138a66129a824 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 7 Jun 2019 17:14:54 -0700 Subject: [PATCH 07/22] Address comments --- .../scala/org/apache/spark/util/Utils.scala | 5 +- .../sql/catalog/v2/StagingTableCatalog.java | 105 ++++++++++ .../catalog/v2/TransactionalTableCatalog.java | 39 ---- .../spark/sql/sources/v2/StagedTable.java | 27 +++ .../logical/sql/ReplaceTableStatement.scala | 9 +- .../sql/catalyst/parser/DDLParserSuite.scala | 2 +- .../datasources/v2/CreateTableExec.scala | 36 +++- .../datasources/v2/DataSourceV2Strategy.scala | 35 +++- .../datasources/v2/ReplaceTableExec.scala | 42 ++-- .../v2/WriteToDataSourceV2Exec.scala | 197 +++++++++++------- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 144 ++++++++----- .../sources/v2/TestInMemoryTableCatalog.scala | 10 +- 12 files changed, 452 insertions(+), 199 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java delete mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TransactionalTableCatalog.java diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index bed50865e7be..5f0c6f0bd4f2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1389,7 +1389,10 @@ private[spark] object Utils extends Logging { originalThrowable = cause try { logError("Aborting task", originalThrowable) - TaskContext.get().markTaskFailed(originalThrowable) + val taskContext = TaskContext.get() + if (taskContext != null) { + taskContext.markTaskFailed(originalThrowable) + } catchBlock } catch { case t: Throwable => diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java new file mode 100644 index 000000000000..0c0559d5ea53 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalog.v2; + +import java.util.Map; + +import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.sources.v2.StagedTable; +import org.apache.spark.sql.sources.v2.SupportsWrite; +import org.apache.spark.sql.sources.v2.writer.BatchWrite; +import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * An optional mix-in for implementations of {@link TableCatalog} that support staging creation of + * the a table before committing the table's metadata along with its contents in CREATE TABLE AS + * SELECT or REPLACE TABLE AS SELECT operations. + *

+ * It is highly recommended to implement this trait whenever possible so that CREATE TABLE AS + * SELECT and REPLACE TABLE AS SELECT operations are atomic. For example, when one runs a REPLACE + * TABLE AS SELECT operation, if the catalog does not implement this trait, the planner will first + * drop the table via {@link TableCatalog#dropTable(Identifier)}, then create the table via + * {@link TableCatalog#createTable(Identifier, StructType, Transform[], Map)}, and then perform + * the write via {@link SupportsWrite#newWriteBuilder(CaseInsensitiveStringMap)}. However, if the + * write operation fails, the catalog will have already dropped the table, and the planner cannot + * roll back the dropping of the table. + *

+ * If the catalog implements this plugin, the catalog can implement the methods to "stage" the + * creation and the replacement of a table. After the table's + * {@link BatchWrite#commit(WriterCommitMessage[])} is called, + * {@link StagedTable#commitStagedChanges()} is called, at which point the staged table can + * complete both the data write and the metadata swap operation atomically. + */ +public interface StagingTableCatalog extends TableCatalog { + + /** + * Stage the creation of a table, preparing it to be committed into the metastore. + *

+ * When the table is committed, the contents of any writes performed by the Spark planner are + * committed along with the metadata about the table passed into this method's arguments. If the + * table exists when this method is called, the method should throw an exception accordingly. If + * another process concurrently creates the table before this table's staged changes are + * committed, an exception should be thrown by {@link StagedTable#commitStagedChanges()}. + * + * @param ident a table identifier + * @param schema the schema of the new table, as a struct type + * @param partitions transforms to use for partitioning data in the table + * @param properties a string map of table properties + * @return metadata for the new table + * @throws TableAlreadyExistsException If a table or view already exists for the identifier + * @throws UnsupportedOperationException If a requested partition transform is not supported + * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + */ + StagedTable stageCreate( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; + ; + + /** + * Stage the replacement of a table, preparing it to be committed into the metastore when the + * returned table's {@link StagedTable#commitStagedChanges()} is called. + *

+ * When the table is committed, the contents of any writes performed by the Spark planner are + * committed along with the metadata about the table passed into this method's arguments. If the + * table exists, the metadata and the contents of this table replace the metadata and contents of + * the existing table. If the table does not exist, it should be created in the metastore. If a + * concurrent process commits changes to the table's data or metadata in the metastore while the + * write is being performed but before the staged changes are committed, the catalog can decide + * whether to move forward with the table replacement anyways or abort the commit operation. + * + * @param ident a table identifier + * @param schema the schema of the new table, as a struct type + * @param partitions transforms to use for partitioning data in the table + * @param properties a string map of table properties + * @return metadata for the new table + * @throws TableAlreadyExistsException If a table or view already exists for the identifier + * @throws UnsupportedOperationException If a requested partition transform is not supported + * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + */ + StagedTable stageReplace( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TransactionalTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TransactionalTableCatalog.java deleted file mode 100644 index 9f27d74e639a..000000000000 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TransactionalTableCatalog.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalog.v2; - -import java.util.Map; - -import org.apache.spark.sql.catalog.v2.expressions.Transform; -import org.apache.spark.sql.sources.v2.StagedTable; -import org.apache.spark.sql.types.StructType; - -public interface TransactionalTableCatalog { - - StagedTable stageCreate( - Identifier ident, - StructType schema, - Transform[] partitions, - Map properties); - - StagedTable stageReplace( - Identifier ident, - StructType schema, - Transform[] partitions, - Map properties); -} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java index 5c5ff5c12c38..b2baa93b146a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java @@ -17,9 +17,36 @@ package org.apache.spark.sql.sources.v2; +import java.util.Map; +import org.apache.spark.sql.catalog.v2.Identifier; +import org.apache.spark.sql.catalog.v2.StagingTableCatalog; +import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * Represents a table which is staged for being committed to the metastore. + *

+ * This is used to implement atomic CREATE TABLE AS SELECT and REPLACE TABLE AS SELECT queries. The + * planner will create one of these via + * {@link StagingTableCatalog#stageCreate(Identifier, StructType, Transform[], Map)} or + * {@link StagingTableCatalog#stageReplace(Identifier, StructType, Transform[], Map)} to prepare the + * table for being written to. This table should usually implement {@link SupportsWrite}. A new + * writer will be constructed via {@link SupportsWrite#newWriteBuilder(CaseInsensitiveStringMap)}, + * and the write will be committed. The job concludes with a call to {@link #commitStagedChanges()}, + * at which point implementations are expected to commit the table's metadata into the metastore + * along with the data that was written by the writes from the write builder this table created. + */ public interface StagedTable extends Table { + /** + * Finalize the creation or replacement of this table. + */ void commitStagedChanges(); + /** + * Abort the changes that were staged, both in metadata and from temporary outputs of this + * table's writers. + */ void abortStagedChanges(); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala index 05ed4a625297..d935af8d63ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala @@ -38,12 +38,7 @@ case class ReplaceTableStatement( provider: String, options: Map[String, String], location: Option[String], - comment: Option[String]) extends ParsedStatement { - - override def output: Seq[Attribute] = Seq.empty - - override def children: Seq[LogicalPlan] = Seq.empty -} + comment: Option[String]) extends ParsedStatement /** * A REPLACE TABLE AS SELECT command, as parsed from SQL. @@ -59,7 +54,5 @@ case class ReplaceTableAsSelectStatement( location: Option[String], comment: Option[String]) extends ParsedStatement { - override def output: Seq[Attribute] = Seq.empty - override def children: Seq[LogicalPlan] = Seq(asSelect) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index d24364ff1d9b..dd84170e2620 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -49,7 +49,7 @@ class DDLParserSuite extends AnalysisTest { test("create/replace table using - schema") { val createSql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" - val replaceSql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" + val replaceSql = "REPLACE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" val expectedTableSpec = TableSpec( Seq("my_tab"), Some(new StructType() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala index f35758bf08c6..07696bf3f63c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -20,13 +20,14 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils case class CreateTableExec( catalog: TableCatalog, @@ -54,3 +55,36 @@ case class CreateTableExec( override def output: Seq[Attribute] = Seq.empty } + +case class CreateTableStagingExec( + catalog: StagingTableCatalog, + identifier: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + tableProperties: Map[String, String], + ignoreIfExists: Boolean) extends LeafExecNode { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + override protected def doExecute(): RDD[InternalRow] = { + if (!catalog.tableExists(identifier)) { + try { + val stagedCreate = catalog.stageCreate( + identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + Utils.tryWithSafeFinallyAndFailureCallbacks({ + stagedCreate.commitStagedChanges() + })(catchBlock = { + stagedCreate.abortStagedChanges() + }) + } catch { + case _: TableAlreadyExistsException if ignoreIfExists => + logWarning(s"Table ${identifier.quoted} was created concurrently. Ignoring.") + } + } else if (!ignoreIfExists) { + throw new TableAlreadyExistsException(identifier) + } + + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 56ca182fd989..4da2a6913525 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} +import org.apache.spark.sql.catalog.v2.StagingTableCatalog import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect} @@ -161,20 +162,42 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil case CreateV2Table(catalog, ident, schema, parts, props, ifNotExists) => - CreateTableExec(catalog, ident, schema, parts, props, ifNotExists) :: Nil + catalog match { + case staging: StagingTableCatalog => + CreateTableStagingExec(staging, ident, schema, parts, props, ifNotExists) :: Nil + case _ => + CreateTableExec(catalog, ident, schema, parts, props, ifNotExists) :: Nil + } case CreateTableAsSelect(catalog, ident, parts, query, props, options, ifNotExists) => val writeOptions = new CaseInsensitiveStringMap(options.asJava) - CreateTableAsSelectExec( - catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + catalog match { + case staging: StagingTableCatalog => + CreateTableAsSelectStagingExec( + staging, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + case _ => + CreateTableAsSelectExec( + catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + } case ReplaceTable(catalog, ident, schema, parts, props) => - ReplaceTableExec(catalog, ident, schema, parts, props) :: Nil + catalog match { + case staging: StagingTableCatalog => + ReplaceTableStagingExec(staging, ident, schema, parts, props) :: Nil + case _ => + ReplaceTableExec(catalog, ident, schema, parts, props) :: Nil + } case ReplaceTableAsSelect(catalog, ident, parts, query, props, options) => val writeOptions = new CaseInsensitiveStringMap(options.asJava) - ReplaceTableAsSelectExec( - catalog, ident, parts, planLater(query), props, writeOptions) :: Nil + catalog match { + case staging: StagingTableCatalog => + ReplaceTableAsSelectStagingExec( + staging, ident, parts, planLater(query), props, writeOptions) :: Nil + case _ => + ReplaceTableAsSelectExec( + catalog, ident, parts, planLater(query), props, writeOptions) :: Nil + } case AppendData(r: DataSourceV2Relation, query, _) => AppendDataExec(r.table.asWritable, r.options, planLater(query)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index 2a348f160e95..f3fa47009049 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TransactionalTableCatalog} +import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils case class ReplaceTableExec( catalog: TableCatalog, @@ -35,19 +36,34 @@ case class ReplaceTableExec( tableProperties: Map[String, String]) extends LeafExecNode { override protected def doExecute(): RDD[InternalRow] = { - catalog match { - case transactional: TransactionalTableCatalog => - transactional.stageReplace( - identifier, - tableSchema, - partitioning.toArray, - tableProperties.asJava).commitStagedChanges() - case _ => - if (catalog.tableExists(identifier)) { - catalog.dropTable(identifier) - } - catalog.createTable(identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + if (catalog.tableExists(identifier)) { + catalog.dropTable(identifier) } + catalog.createTable(identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } + + override def output: Seq[Attribute] = Seq.empty +} + +case class ReplaceTableStagingExec( + catalog: StagingTableCatalog, + identifier: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + tableProperties: Map[String, String]) extends LeafExecNode { + + override protected def doExecute(): RDD[InternalRow] = { + val stagedReplace = catalog.stageReplace( + identifier, + tableSchema, + partitioning.toArray, + tableProperties.asJava) + Utils.tryWithSafeFinallyAndFailureCallbacks({ + stagedReplace.commitStagedChanges() + })(catchBlock = { + stagedReplace.abortStagedChanges() + }) sqlContext.sparkContext.parallelize(Seq.empty, 1) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index c38ab5821da8..411e3a08c6f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -26,7 +26,7 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.executor.CommitDeniedException import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TransactionalTableCatalog} +import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException @@ -51,14 +51,13 @@ case class WriteToDataSourceV2(batchWrite: BatchWrite, query: LogicalPlan) } /** - * Physical plan node for v2 create table as select. + * Physical plan node for v2 create table as select when the catalog does not support staging + * the table creation. * * A new table will be created using the schema of the query, and rows from the query are appended. - * If either table creation or the append fails, the table will be deleted. The CTAS operation is - * atomic if the catalog supports atomic operations, where the creation of the table is staged and - * the commit of the write should bundle the commitment of the metadata and the table contents in - * a single unit. Otherwise, the table is created, and if the write fails, the table is - * subsequently dropped. + * If either table creation or the append fails, the table will be deleted. This implementation is + * not atomic; for an atomic variant for catalogs that support the appropriate features, see + * CreateTableAsSelectStagingExec. */ case class CreateTableAsSelectExec( catalog: TableCatalog, @@ -67,7 +66,7 @@ case class CreateTableAsSelectExec( query: SparkPlan, properties: Map[String, String], writeOptions: CaseInsensitiveStringMap, - ifNotExists: Boolean) extends StagedTableWriteExec { + ifNotExists: Boolean) extends V2TableWriteExec { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper @@ -79,44 +78,69 @@ case class CreateTableAsSelectExec( throw new TableAlreadyExistsException(ident) } - catalog match { - case txnCatalog: TransactionalTableCatalog => - val stagedTable = txnCatalog.stageCreate( - ident, query.schema, partitioning.toArray, properties.asJava) - writeToStagedTable(stagedTable, writeOptions, ident) - case _ => - Utils.tryWithSafeFinallyAndFailureCallbacks({ - catalog.createTable( - ident, query.schema, partitioning.toArray, properties.asJava) match { - case table: SupportsWrite => - val batchWrite = table.newWriteBuilder(writeOptions) - .withInputDataSchema(query.schema) - .withQueryId(UUID.randomUUID().toString) - .buildForBatch() - - doWrite(batchWrite) - - case _ => - // table does not support writes - throw new SparkException( - s"Table implementation does not support writes: ${ident.quoted}") - } - })(catchBlock = { - catalog.dropTable(ident) - }) + Utils.tryWithSafeFinallyAndFailureCallbacks({ + catalog.createTable( + ident, query.schema, partitioning.toArray, properties.asJava) match { + case table: SupportsWrite => + val batchWrite = table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + .buildForBatch() + + doWrite(batchWrite) + + case _ => + // table does not support writes + throw new SparkException( + s"Table implementation does not support writes: ${ident.quoted}") + } + })(catchBlock = { + catalog.dropTable(ident) + }) + } +} + +/** + * Physical plan node for v2 create table as select, when the catalog is determined to support + * staging table creation. + * + * A new table will be created using the schema of the query, and rows from the query are appended. + * The CTAS operation is atomic. The creation of the table is staged and the commit of the write + * should bundle the commitment of the metadata and the table contents in a single unit. If the + * write fails, the table is instructed to roll back all staged changes. + */ +case class CreateTableAsSelectStagingExec ( + catalog: StagingTableCatalog, + ident: Identifier, + partitioning: Seq[Transform], + query: SparkPlan, + properties: Map[String, String], + writeOptions: CaseInsensitiveStringMap, + ifNotExists: Boolean) extends StagedTableWriteExec { + + override protected def doExecute(): RDD[InternalRow] = { + if (catalog.tableExists(ident)) { + if (ifNotExists) { + return sparkContext.parallelize(Seq.empty, 1) + } + + throw new TableAlreadyExistsException(ident) } + val stagedTable = catalog.stageCreate( + ident, query.schema, partitioning.toArray, properties.asJava) + writeToStagedTable(stagedTable, writeOptions, ident) } } /** - * Physical plan node for v2 replace table as select. + * Physical plan node for v2 replace table as select when the catalog does not support staging + * table replacement. * * A new table will be created using the schema of the query, and rows from the query are appended. * If the table exists, its contents and schema should be replaced with the schema and the contents - * of the query. If the catalog supports atomic replacement of tables, the plugin's transactional - * implementation is used so that the plugin can avoid losing previously committed data if any - * part of the write fails. The non-atomic implementation will drop the table and then run - * non-atomic CTAS. + * of the query. This is a non-atomic implementation that drops the table and then runs non-atomic + * CTAS. For an atomic implementation for catalogs with the appropriate support, see + * ReplaceTableAsSelectStagingExec. */ case class ReplaceTableAsSelectExec( catalog: TableCatalog, @@ -129,44 +153,64 @@ case class ReplaceTableAsSelectExec( import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper override protected def doExecute(): RDD[InternalRow] = { - catalog match { - case txnCatalog: TransactionalTableCatalog => - val stagedTable = txnCatalog.stageReplace( - ident, query.schema, partitioning.toArray, properties.asJava) - writeToStagedTable(stagedTable, writeOptions, ident) - case _ => - // Note that this operation is potentially unsafe, but these are the strict semantics of - // RTAS if the catalog does not support atomic operations. - // - // There are numerous cases we concede to where the table will be dropped and irrecoverable: - // - // 1. Creating the new table fails, - // 2. Writing to the new table fails, - // 3. The table returned by catalog.createTable doesn't support writing. - if (catalog.tableExists(ident)) { - catalog.dropTable(ident) - } - - Utils.tryWithSafeFinallyAndFailureCallbacks({ - catalog.createTable( - ident, query.schema, partitioning.toArray, properties.asJava) match { - case table: SupportsWrite => - val batchWrite = table.newWriteBuilder(writeOptions) - .withInputDataSchema(query.schema) - .withQueryId(UUID.randomUUID().toString) - .buildForBatch() - - doWrite(batchWrite) - - case _ => - // table does not support writes - throw new SparkException( - s"Table implementation does not support writes: ${ident.quoted}") - } - })(catchBlock = { - catalog.dropTable(ident) - }) + // Note that this operation is potentially unsafe, but these are the strict semantics of + // RTAS if the catalog does not support atomic operations. + // + // There are numerous cases we concede to where the table will be dropped and irrecoverable: + // + // 1. Creating the new table fails, + // 2. Writing to the new table fails, + // 3. The table returned by catalog.createTable doesn't support writing. + if (catalog.tableExists(ident)) { + catalog.dropTable(ident) } + val createdTable = catalog.createTable( + ident, query.schema, partitioning.toArray, properties.asJava) + Utils.tryWithSafeFinallyAndFailureCallbacks({ + createdTable match { + case table: SupportsWrite => + val batchWrite = table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + .buildForBatch() + + doWrite(batchWrite) + + case _ => + // table does not support writes + throw new SparkException( + s"Table implementation does not support writes: ${ident.quoted}") + } + })(catchBlock = { + catalog.dropTable(ident) + }) + } +} + +/** + * + * Physical plan node for v2 replace table as select when the catalog supports staging + * table replacement. + * + * A new table will be created using the schema of the query, and rows from the query are appended. + * If the table exists, its contents and schema should be replaced with the schema and the contents + * of the query. This implementation is atomic. The table replacement is staged, and the commit + * operation at the end should perform tne replacement of the table's metadata and contents. If the + * write fails, the table is instructed to roll back staged changes and any previously written table + * is left untouched. + */ +case class ReplaceTableAsSelectStagingExec( + catalog: StagingTableCatalog, + ident: Identifier, + partitioning: Seq[Transform], + query: SparkPlan, + properties: Map[String, String], + writeOptions: CaseInsensitiveStringMap) extends StagedTableWriteExec { + + override protected def doExecute(): RDD[InternalRow] = { + val stagedTable = catalog.stageReplace( + ident, query.schema, partitioning.toArray, properties.asJava) + writeToStagedTable(stagedTable, writeOptions, ident) } } @@ -421,11 +465,12 @@ trait StagedTableWriteExec extends V2TableWriteExec { stagedTable.commitStagedChanges() writtenRows case _ => - // table does not support writes + // Table does not support writes - staged changes are also rolled back below. throw new SparkException( s"Table implementation does not support writes: ${ident.quoted}") } })(catchBlock = { + // Failure rolls back the staged writes and metadata changes. stagedTable.abortStagedChanges() }) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index 63c3cdd076a7..abae1b0a88db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -37,7 +37,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn before { spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) spark.conf.set( - "spark.sql.catalog.testcatatomic", classOf[TestTransactionalInMemoryCatalog].getName) + "spark.sql.catalog.testcat_atomic", classOf[TestStagingInMemoryCatalog].getName) spark.conf.set("spark.sql.default.catalog", "testcat") val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") @@ -48,7 +48,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn after { spark.catalog("testcat").asInstanceOf[TestInMemoryTableCatalog].clearTables() - spark.catalog("testcatatomic").asInstanceOf[TestInMemoryTableCatalog].clearTables() + spark.catalog("testcat_atomic").asInstanceOf[TestInMemoryTableCatalog].clearTables() spark.sql("DROP TABLE source") } @@ -157,55 +157,71 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn } test("CreateTableAsSelect: use v2 plan because catalog is set") { - spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") - - val testCatalog = spark.catalog("testcat").asTableCatalog - val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) - - assert(table.name == "testcat.table_name") - assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) - assert(table.schema == new StructType() - .add("id", LongType, nullable = false) - .add("data", StringType)) - - val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) - checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + val basicCatalog = spark.catalog("testcat").asTableCatalog + val atomicCatalog = spark.catalog("testcat_atomic").asTableCatalog + val basicIdentifier = "testcat.table_name" + val atomicIdentifier = "testcat_atomic.table_name" + + Seq((basicCatalog, basicIdentifier), (atomicCatalog, atomicIdentifier)).foreach { + case (catalog, identifier) => + spark.sql(s"CREATE TABLE $identifier USING foo AS SELECT id, data FROM source") + + val table = catalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == identifier) + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + } } - test("ReplaceTableAsSelect: basic v2 implementation using atomic catalog.") { - spark.sql("CREATE TABLE testcatatomic.table_name USING foo AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcatatomic").asTableCatalog - val originalTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) - spark.sql("REPLACE TABLE testcatatomic.table_name USING foo AS SELECT id FROM source") - val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) - assert(replacedTable != originalTable, "Table should have been replaced.") - assert(replacedTable.name == "testcatatomic.table_name") - assert(replacedTable.partitioning.isEmpty) - assert(replacedTable.properties == Map("provider" -> "foo").asJava) - assert(replacedTable.schema == new StructType() - .add("id", LongType, nullable = false)) - val rdd = spark.sparkContext.parallelize(replacedTable.asInstanceOf[InMemoryTable].rows) - checkAnswer( - spark.internalCreateDataFrame(rdd, replacedTable.schema), - spark.table("source").select("id")) + test("ReplaceTableAsSelect: basic v2 implementation.") { + val basicCatalog = spark.catalog("testcat").asTableCatalog + val atomicCatalog = spark.catalog("testcat_atomic").asTableCatalog + val basicIdentifier = "testcat.table_name" + val atomicIdentifier = "testcat_atomic.table_name" + + Seq((basicCatalog, basicIdentifier), (atomicCatalog, atomicIdentifier)).foreach { + case (catalog, identifier) => + spark.sql(s"CREATE TABLE $identifier USING foo AS SELECT id, data FROM source") + val originalTable = catalog.loadTable(Identifier.of(Array(), "table_name")) + + spark.sql(s"REPLACE TABLE $identifier USING foo AS SELECT id FROM source") + val replacedTable = catalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(replacedTable != originalTable, "Table should have been replaced.") + assert(replacedTable.name == identifier) + assert(replacedTable.partitioning.isEmpty) + assert(replacedTable.properties == Map("provider" -> "foo").asJava) + assert(replacedTable.schema == new StructType() + .add("id", LongType, nullable = false)) + + val rdd = spark.sparkContext.parallelize(replacedTable.asInstanceOf[InMemoryTable].rows) + checkAnswer( + spark.internalCreateDataFrame(rdd, replacedTable.schema), + spark.table("source").select("id")) + } } - test("ReplaceTableAsSelect: Non-atomic catalog creates the empty table, but leaves the" + - " table empty if the write fails.") { + test("ReplaceTableAsSelect: Non-atomic catalog drops the table if the write fails.") { spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") val testCatalog = spark.catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) + intercept[Exception] { spark.sql("REPLACE TABLE testcat.table_name" + s" USING foo OPTIONS (`${TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION}`=true)" + s" AS SELECT id FROM source") } - val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) - assert(replacedTable != table, "Table should have been replaced.") - assert(replacedTable.asInstanceOf[InMemoryTable].rows.isEmpty, - "Rows should not have been successfully written to the replaced table.") + + assert(!testCatalog.tableExists(Identifier.of(Array(), "table_name")), + "Table should have been dropped as a result of the replace.") } test("ReplaceTableAsSelect: Non-atomic catalog drops the table permanently if the" + @@ -214,44 +230,74 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn val testCatalog = spark.catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) + intercept[Exception] { spark.sql("REPLACE TABLE testcat.table_name" + s" USING foo" + s" TBLPROPERTIES (`${TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY}`=true)" + s" AS SELECT id FROM source") } + assert(!testCatalog.tableExists(Identifier.of(Array(), "table_name")), "Table should have been dropped and failed to be created.") } test("ReplaceTableAsSelect: Atomic catalog does not drop the table when replace fails.") { - spark.sql("CREATE TABLE testcatatomic.table_name USING foo AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcatatomic").asTableCatalog + spark.sql("CREATE TABLE testcat_atomic.table_name USING foo AS SELECT id, data FROM source") + val testCatalog = spark.catalog("testcat_atomic").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) - assert(table.name == "testcatatomic.table_name") - assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) - assert(table.schema == new StructType() - .add("id", LongType, nullable = false) - .add("data", StringType)) intercept[Exception] { - spark.sql("REPLACE TABLE testcatatomic.table_name" + + spark.sql("REPLACE TABLE testcat_atomic.table_name" + s" USING foo OPTIONS (`${TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION}=true)" + s" AS SELECT id FROM source") } - var maybeReplacedTable: Table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + var maybeReplacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(maybeReplacedTable === table, "Table should not have changed.") + intercept[Exception] { - spark.sql("REPLACE TABLE testcatatomic.table_name" + + spark.sql("REPLACE TABLE testcat_atomic.table_name" + s" USING foo" + s" TBLPROPERTIES (`${TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY}`=true)" + s" AS SELECT id FROM source") } + maybeReplacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(maybeReplacedTable === table, "Table should not have changed.") } + test("ReplaceTable: Erases the table contents and changes the metadata.") { + spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) + + spark.sql("REPLACE TABLE testcat.table_name (id bigint) USING foo") + val replaced = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(replaced.asInstanceOf[InMemoryTable].rows.isEmpty, + "Replaced table should have no rows after committing.") + assert(replaced.schema().fields.length === 1, + "Replaced table should have new schema.") + assert(replaced.schema().fields(0).name === "id", + "Replaced table should have new schema.") + } + + test("ReplaceTableAsSelect: New table has same behavior as CTAS.") { + spark.sql(s"CREATE TABLE created USING $orc2 AS SELECT id, data FROM source") + spark.sql(s"REPLACE TABLE replaced USING $orc2 AS SELECT id, data FROM source") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val createdTable = testCatalog.loadTable(Identifier.of(Array(), "created")) + val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "replaced")) + + assert(createdTable.asInstanceOf[InMemoryTable].rows === + replacedTable.asInstanceOf[InMemoryTable].rows) + assert(createdTable.schema === replacedTable.schema) + } + test("CreateTableAsSelect: use v2 plan because provider is v2") { spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 2185691f62f5..6d13397e55d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -23,7 +23,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, TableCatalog, TableChange, TestTableCatalog, TransactionalTableCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, StagingTableCatalog, TableCatalog, TableChange} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.InternalRow @@ -211,15 +211,15 @@ object TestInMemoryTableCatalog { } } -class TestTransactionalInMemoryCatalog - extends TestInMemoryTableCatalog with TransactionalTableCatalog { +class TestStagingInMemoryCatalog + extends TestInMemoryTableCatalog with StagingTableCatalog { override def stageCreate( ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - newStagedTable(ident, schema, partitions, properties, false) + newStagedTable(ident, schema, partitions, properties, replaceIfExists = false) } override def stageReplace( @@ -227,7 +227,7 @@ class TestTransactionalInMemoryCatalog schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - newStagedTable(ident, schema, partitions, properties, true) + newStagedTable(ident, schema, partitions, properties, replaceIfExists = true) } private def newStagedTable( From a9142e9278e61121f279d1aa7bef69ef863e3b36 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 7 Jun 2019 17:49:46 -0700 Subject: [PATCH 08/22] Fix javadoc --- .../org/apache/spark/sql/catalog/v2/StagingTableCatalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java index 0c0559d5ea53..97cd6c30c51c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java @@ -101,5 +101,5 @@ StagedTable stageReplace( Identifier ident, StructType schema, Transform[] partitions, - Map properties); + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; } From 842886e061c558bf899e031487d6a44c72fd508d Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 14 Jun 2019 18:46:18 -0700 Subject: [PATCH 09/22] Address comments --- .../datasources/v2/CreateTableExec.scala | 34 ------------------- .../datasources/v2/DataSourceV2Strategy.scala | 9 ++--- .../v2/WriteToDataSourceV2Exec.scala | 3 +- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 18 +++++----- 4 files changed, 14 insertions(+), 50 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala index 07696bf3f63c..b6f741954787 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.types.StructType -import org.apache.spark.util.Utils case class CreateTableExec( catalog: TableCatalog, @@ -55,36 +54,3 @@ case class CreateTableExec( override def output: Seq[Attribute] = Seq.empty } - -case class CreateTableStagingExec( - catalog: StagingTableCatalog, - identifier: Identifier, - tableSchema: StructType, - partitioning: Seq[Transform], - tableProperties: Map[String, String], - ignoreIfExists: Boolean) extends LeafExecNode { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ - - override protected def doExecute(): RDD[InternalRow] = { - if (!catalog.tableExists(identifier)) { - try { - val stagedCreate = catalog.stageCreate( - identifier, tableSchema, partitioning.toArray, tableProperties.asJava) - Utils.tryWithSafeFinallyAndFailureCallbacks({ - stagedCreate.commitStagedChanges() - })(catchBlock = { - stagedCreate.abortStagedChanges() - }) - } catch { - case _: TableAlreadyExistsException if ignoreIfExists => - logWarning(s"Table ${identifier.quoted} was created concurrently. Ignoring.") - } - } else if (!ignoreIfExists) { - throw new TableAlreadyExistsException(identifier) - } - - sqlContext.sparkContext.parallelize(Seq.empty, 1) - } - - override def output: Seq[Attribute] = Seq.empty -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 4da2a6913525..cfc9095c22d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -162,18 +162,13 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil case CreateV2Table(catalog, ident, schema, parts, props, ifNotExists) => - catalog match { - case staging: StagingTableCatalog => - CreateTableStagingExec(staging, ident, schema, parts, props, ifNotExists) :: Nil - case _ => - CreateTableExec(catalog, ident, schema, parts, props, ifNotExists) :: Nil - } + CreateTableExec(catalog, ident, schema, parts, props, ifNotExists) :: Nil case CreateTableAsSelect(catalog, ident, parts, query, props, options, ifNotExists) => val writeOptions = new CaseInsensitiveStringMap(options.asJava) catalog match { case staging: StagingTableCatalog => - CreateTableAsSelectStagingExec( + AtomicCreateTableAsSelectExec( staging, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil case _ => CreateTableAsSelectExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 411e3a08c6f9..e68b983e64d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -77,6 +77,7 @@ case class CreateTableAsSelectExec( } throw new TableAlreadyExistsException(ident) + } Utils.tryWithSafeFinallyAndFailureCallbacks({ catalog.createTable( @@ -109,7 +110,7 @@ case class CreateTableAsSelectExec( * should bundle the commitment of the metadata and the table contents in a single unit. If the * write fails, the table is instructed to roll back all staged changes. */ -case class CreateTableAsSelectStagingExec ( +case class AtomicCreateTableAsSelectExec( catalog: StagingTableCatalog, ident: Identifier, partitioning: Seq[Transform], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index abae1b0a88db..16dcfbfb6354 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -286,16 +286,18 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn } test("ReplaceTableAsSelect: New table has same behavior as CTAS.") { - spark.sql(s"CREATE TABLE created USING $orc2 AS SELECT id, data FROM source") - spark.sql(s"REPLACE TABLE replaced USING $orc2 AS SELECT id, data FROM source") + Seq("testcat", "testcat_atomic").foreach { catalog => + spark.sql(s"CREATE TABLE $catalog.created USING $orc2 AS SELECT id, data FROM source") + spark.sql(s"REPLACE TABLE $catalog.replaced USING $orc2 AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcat").asTableCatalog - val createdTable = testCatalog.loadTable(Identifier.of(Array(), "created")) - val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "replaced")) + val testCatalog = spark.catalog(catalog).asTableCatalog + val createdTable = testCatalog.loadTable(Identifier.of(Array(), "created")) + val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "replaced")) - assert(createdTable.asInstanceOf[InMemoryTable].rows === - replacedTable.asInstanceOf[InMemoryTable].rows) - assert(createdTable.schema === replacedTable.schema) + assert(createdTable.asInstanceOf[InMemoryTable].rows === + replacedTable.asInstanceOf[InMemoryTable].rows) + assert(createdTable.schema === replacedTable.schema) + } } test("CreateTableAsSelect: use v2 plan because provider is v2") { From 2bf4b5fb1c5c9480bba59627a7b61865619f5502 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 14 Jun 2019 18:47:16 -0700 Subject: [PATCH 10/22] Fix merge conflicts --- .../apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index 4f9ceba124ce..5470327043ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -36,13 +36,9 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn before { spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) -<<<<<<< HEAD spark.conf.set( "spark.sql.catalog.testcat_atomic", classOf[TestStagingInMemoryCatalog].getName) -||||||| merged common ancestors -======= spark.conf.set("spark.sql.catalog.testcat2", classOf[TestInMemoryTableCatalog].getName) ->>>>>>> origin/master spark.conf.set("spark.sql.default.catalog", "testcat") val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") From 80dc0cc1ae7e8888f1a9d05b7b834ff7e555d753 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 25 Jun 2019 14:45:23 -0700 Subject: [PATCH 11/22] Address comments --- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../datasources/v2/WriteToDataSourceV2Exec.scala | 11 +++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index cfc9095c22d0..4e52eb53ec96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -187,7 +187,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { val writeOptions = new CaseInsensitiveStringMap(options.asJava) catalog match { case staging: StagingTableCatalog => - ReplaceTableAsSelectStagingExec( + AtomicReplaceTableAsSelectExec( staging, ident, parts, planLater(query), props, writeOptions) :: Nil case _ => ReplaceTableAsSelectExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index e68b983e64d5..1c6852b8ffd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -77,7 +77,6 @@ case class CreateTableAsSelectExec( } throw new TableAlreadyExistsException(ident) - } Utils.tryWithSafeFinallyAndFailureCallbacks({ catalog.createTable( @@ -117,7 +116,7 @@ case class AtomicCreateTableAsSelectExec( query: SparkPlan, properties: Map[String, String], writeOptions: CaseInsensitiveStringMap, - ifNotExists: Boolean) extends StagedTableWriteExec { + ifNotExists: Boolean) extends AtomicTableWriteExec { override protected def doExecute(): RDD[InternalRow] = { if (catalog.tableExists(ident)) { @@ -149,7 +148,7 @@ case class ReplaceTableAsSelectExec( partitioning: Seq[Transform], query: SparkPlan, properties: Map[String, String], - writeOptions: CaseInsensitiveStringMap) extends StagedTableWriteExec { + writeOptions: CaseInsensitiveStringMap) extends AtomicTableWriteExec { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper @@ -200,13 +199,13 @@ case class ReplaceTableAsSelectExec( * write fails, the table is instructed to roll back staged changes and any previously written table * is left untouched. */ -case class ReplaceTableAsSelectStagingExec( +case class AtomicReplaceTableAsSelectExec( catalog: StagingTableCatalog, ident: Identifier, partitioning: Seq[Transform], query: SparkPlan, properties: Map[String, String], - writeOptions: CaseInsensitiveStringMap) extends StagedTableWriteExec { + writeOptions: CaseInsensitiveStringMap) extends AtomicTableWriteExec { override protected def doExecute(): RDD[InternalRow] = { val stagedTable = catalog.stageReplace( @@ -447,7 +446,7 @@ object DataWritingSparkTask extends Logging { } } -trait StagedTableWriteExec extends V2TableWriteExec { +private[v2] trait AtomicTableWriteExec extends V2TableWriteExec { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper protected def writeToStagedTable( From 4011a8be71dba3cda3b725b697d31967f9250edd Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 3 Jul 2019 14:30:13 -0700 Subject: [PATCH 12/22] Resolve conflict --- core/src/main/scala/org/apache/spark/util/Utils.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 647d13e287ef..80d70a1d4850 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1389,18 +1389,9 @@ private[spark] object Utils extends Logging { originalThrowable = cause try { logError("Aborting task", originalThrowable) -<<<<<<< HEAD - val taskContext = TaskContext.get() - if (taskContext != null) { - taskContext.markTaskFailed(originalThrowable) - } -||||||| merged common ancestors - TaskContext.get().markTaskFailed(originalThrowable) -======= if (TaskContext.get() != null) { TaskContext.get().markTaskFailed(originalThrowable) } ->>>>>>> origin/master catchBlock } catch { case t: Throwable => From aebf76731997de7569f81884a5a1a6408b43d2b9 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 3 Jul 2019 14:37:48 -0700 Subject: [PATCH 13/22] Newline --- .../sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 1c6852b8ffd3..e0daf1cf6978 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -78,6 +78,7 @@ case class CreateTableAsSelectExec( throw new TableAlreadyExistsException(ident) } + Utils.tryWithSafeFinallyAndFailureCallbacks({ catalog.createTable( ident, query.schema, partitioning.toArray, properties.asJava) match { From 6231f6a8f391c59c06868d275f5209d5fc39678b Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 8 Jul 2019 14:31:13 -0700 Subject: [PATCH 14/22] Add support for CREATE OR REPLACE TABLE --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../CannotReplaceMissingTableException.scala | 28 +++++++++++ .../sql/catalyst/parser/AstBuilder.scala | 9 ++-- .../plans/logical/basicLogicalOperators.scala | 6 ++- .../logical/sql/ReplaceTableStatement.scala | 6 ++- .../datasources/DataSourceResolution.scala | 10 ++-- .../datasources/v2/DataSourceV2Strategy.scala | 12 ++--- .../datasources/v2/ReplaceTableExec.scala | 46 +++++++++++++------ .../v2/WriteToDataSourceV2Exec.scala | 23 +++++++--- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 16 +++++-- 10 files changed, 117 insertions(+), 41 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala 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 12dc1acbf6b3..13c6e3aa7a8b 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 @@ -270,7 +270,7 @@ createTableHeader ; replaceTableHeader - : REPLACE TEMPORARY? TABLE multipartIdentifier + : (CREATE OR)? REPLACE TEMPORARY? TABLE multipartIdentifier ; bucketSpec diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala new file mode 100644 index 000000000000..6c05b4f10f0c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.Identifier + +class CannotReplaceMissingTableException( + tableIdentifier: Identifier) + extends AnalysisException( + s"Table $tableIdentifier cannot be replaced as it did not exist." + + s" Use CREATE OR REPLACE TABLE to create the table.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 942dc47e50de..648c9943ad56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2339,6 +2339,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val provider = ctx.tableProvider.qualifiedName.getText val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) val comment = Option(ctx.comment).map(string) + val orCreate = ctx.replaceTableHeader().CREATE() != null Option(ctx.query).map(plan) match { case Some(_) if temp => @@ -2350,8 +2351,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx) case Some(query) => - ReplaceTableAsSelectStatement( - table, query, partitioning, bucketSpec, properties, provider, options, location, comment) + ReplaceTableAsSelectStatement(table, query, partitioning, bucketSpec, properties, + provider, options, location, comment, orCreate = orCreate) case None if temp => // REPLACE TEMPORARY TABLE ... USING ... is not supported by the catalyst parser. @@ -2359,8 +2360,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging operationNotAllowed("REPLACE TEMPORARY TABLE IF NOT EXISTS", ctx) case _ => - ReplaceTableStatement(table, schema.getOrElse(new StructType), partitioning, bucketSpec, - properties, provider, options, location, comment) + ReplaceTableStatement(table, schema.getOrElse(new StructType), partitioning, + bucketSpec, properties, provider, options, location, comment, orCreate = orCreate) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 1eb209ee1fc6..37730c400f0e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -451,7 +451,8 @@ case class ReplaceTable( tableName: Identifier, tableSchema: StructType, partitioning: Seq[Transform], - properties: Map[String, String]) extends Command + properties: Map[String, String], + orCreate: Boolean) extends Command /** * Replaces a table from a select query with a v2 catalog. @@ -464,7 +465,8 @@ case class ReplaceTableAsSelect( partitioning: Seq[Transform], query: LogicalPlan, properties: Map[String, String], - writeOptions: Map[String, String]) extends Command { + writeOptions: Map[String, String], + orCreate: Boolean) extends Command { override def children: Seq[LogicalPlan] = Seq(query) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala index d935af8d63ce..2808892b089b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala @@ -38,7 +38,8 @@ case class ReplaceTableStatement( provider: String, options: Map[String, String], location: Option[String], - comment: Option[String]) extends ParsedStatement + comment: Option[String], + orCreate: Boolean) extends ParsedStatement /** * A REPLACE TABLE AS SELECT command, as parsed from SQL. @@ -52,7 +53,8 @@ case class ReplaceTableAsSelectStatement( provider: String, options: Map[String, String], location: Option[String], - comment: Option[String]) extends ParsedStatement { + comment: Option[String], + orCreate: Boolean) extends ParsedStatement { override def children: Seq[LogicalPlan] = Seq(asSelect) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index c7589da7405e..150d32f0effa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -87,14 +87,14 @@ case class DataSourceResolution( case ReplaceTableStatement( AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, - V1WriteProvider(provider), options, location, comment) => + V1WriteProvider(provider), options, location, comment, orCreate) => throw new AnalysisException( s"Replacing tables is not supported using the legacy / v1 Spark external catalog" + s" API. Write provider name: $provider, identifier: $table.") case ReplaceTableAsSelectStatement( AsTableIdentifier(table), query, partitionCols, bucketSpec, properties, - V1WriteProvider(provider), options, location, comment) => + V1WriteProvider(provider), options, location, comment, orCreate) => throw new AnalysisException( s"Replacing tables is not supported using the legacy / v1 Spark external catalog" + s" API. Write provider name: $provider, identifier: $table.") @@ -261,7 +261,8 @@ case class DataSourceResolution( partitioning, rtas.asSelect, properties, - writeOptions = rtas.options.filterKeys(_ != "path")) + writeOptions = rtas.options.filterKeys(_ != "path"), + orCreate = rtas.orCreate) } private def convertReplaceTable( @@ -278,7 +279,8 @@ case class DataSourceResolution( identifier, replace.tableSchema, partitioning, - properties) + properties, + orCreate = replace.orCreate) } private def convertTableProperties( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 4e52eb53ec96..c928892a94e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -175,23 +175,23 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil } - case ReplaceTable(catalog, ident, schema, parts, props) => + case ReplaceTable(catalog, ident, schema, parts, props, orCreate) => catalog match { case staging: StagingTableCatalog => - ReplaceTableStagingExec(staging, ident, schema, parts, props) :: Nil + AtomicReplaceTableExec(staging, ident, schema, parts, props, orCreate) :: Nil case _ => - ReplaceTableExec(catalog, ident, schema, parts, props) :: Nil + ReplaceTableExec(catalog, ident, schema, parts, props, orCreate) :: Nil } - case ReplaceTableAsSelect(catalog, ident, parts, query, props, options) => + case ReplaceTableAsSelect(catalog, ident, parts, query, props, options, orCreate) => val writeOptions = new CaseInsensitiveStringMap(options.asJava) catalog match { case staging: StagingTableCatalog => AtomicReplaceTableAsSelectExec( - staging, ident, parts, planLater(query), props, writeOptions) :: Nil + staging, ident, parts, planLater(query), props, writeOptions, orCreate) :: Nil case _ => ReplaceTableAsSelectExec( - catalog, ident, parts, planLater(query), props, writeOptions) :: Nil + catalog, ident, parts, planLater(query), props, writeOptions, orCreate) :: Nil } case AppendData(r: DataSourceV2Relation, query, _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index f3fa47009049..cc22aa36dd5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -23,8 +23,10 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.sources.v2.StagedTable import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -33,11 +35,14 @@ case class ReplaceTableExec( identifier: Identifier, tableSchema: StructType, partitioning: Seq[Transform], - tableProperties: Map[String, String]) extends LeafExecNode { + tableProperties: Map[String, String], + orCreate: Boolean) extends LeafExecNode { override protected def doExecute(): RDD[InternalRow] = { if (catalog.tableExists(identifier)) { catalog.dropTable(identifier) + } else if (!orCreate) { + throw new CannotReplaceMissingTableException(identifier) } catalog.createTable(identifier, tableSchema, partitioning.toArray, tableProperties.asJava) sqlContext.sparkContext.parallelize(Seq.empty, 1) @@ -46,27 +51,42 @@ case class ReplaceTableExec( override def output: Seq[Attribute] = Seq.empty } -case class ReplaceTableStagingExec( +case class AtomicReplaceTableExec( catalog: StagingTableCatalog, identifier: Identifier, tableSchema: StructType, partitioning: Seq[Transform], - tableProperties: Map[String, String]) extends LeafExecNode { + tableProperties: Map[String, String], + orCreate: Boolean) extends LeafExecNode { override protected def doExecute(): RDD[InternalRow] = { - val stagedReplace = catalog.stageReplace( - identifier, - tableSchema, - partitioning.toArray, - tableProperties.asJava) - Utils.tryWithSafeFinallyAndFailureCallbacks({ - stagedReplace.commitStagedChanges() - })(catchBlock = { - stagedReplace.abortStagedChanges() - }) + val staged = if (catalog.tableExists(identifier)) { + catalog.stageReplace( + identifier, + tableSchema, + partitioning.toArray, + tableProperties.asJava) + } else if (orCreate) { + catalog.stageCreate( + identifier, + tableSchema, + partitioning.toArray, + tableProperties.asJava) + } else { + throw new CannotReplaceMissingTableException(identifier) + } + commitOrAbortStagedChanges(staged) sqlContext.sparkContext.parallelize(Seq.empty, 1) } override def output: Seq[Attribute] = Seq.empty + + private def commitOrAbortStagedChanges(staged: StagedTable): Unit = { + Utils.tryWithSafeFinallyAndFailureCallbacks({ + staged.commitStagedChanges() + })(catchBlock = { + staged.abortStagedChanges() + }) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index e0daf1cf6978..e26ee74b9baf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -29,7 +29,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} @@ -149,7 +149,8 @@ case class ReplaceTableAsSelectExec( partitioning: Seq[Transform], query: SparkPlan, properties: Map[String, String], - writeOptions: CaseInsensitiveStringMap) extends AtomicTableWriteExec { + writeOptions: CaseInsensitiveStringMap, + orCreate: Boolean) extends AtomicTableWriteExec { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper @@ -163,7 +164,9 @@ case class ReplaceTableAsSelectExec( // 2. Writing to the new table fails, // 3. The table returned by catalog.createTable doesn't support writing. if (catalog.tableExists(ident)) { - catalog.dropTable(ident) + catalog.dropTable(ident) + } else if (!orCreate) { + throw new CannotReplaceMissingTableException(ident) } val createdTable = catalog.createTable( ident, query.schema, partitioning.toArray, properties.asJava) @@ -206,11 +209,19 @@ case class AtomicReplaceTableAsSelectExec( partitioning: Seq[Transform], query: SparkPlan, properties: Map[String, String], - writeOptions: CaseInsensitiveStringMap) extends AtomicTableWriteExec { + writeOptions: CaseInsensitiveStringMap, + orCreate: Boolean) extends AtomicTableWriteExec { override protected def doExecute(): RDD[InternalRow] = { - val stagedTable = catalog.stageReplace( - ident, query.schema, partitioning.toArray, properties.asJava) + val stagedTable = if (catalog.tableExists(ident)) { + catalog.stageReplace( + ident, query.schema, partitioning.toArray, properties.asJava) + } else if (orCreate) { + catalog.stageCreate( + ident, query.schema, partitioning.toArray, properties.asJava) + } else { + throw new CannotReplaceMissingTableException(ident) + } writeToStagedTable(stagedTable, writeOptions, ident) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index 5470327043ed..d6fc39cc06cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -23,7 +23,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalog.v2.Identifier -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{LongType, StringType, StructType} @@ -287,10 +287,11 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn "Replaced table should have new schema.") } - test("ReplaceTableAsSelect: New table has same behavior as CTAS.") { + test("ReplaceTableAsSelect: CREATE OR REPLACE new table has same behavior as CTAS.") { Seq("testcat", "testcat_atomic").foreach { catalog => spark.sql(s"CREATE TABLE $catalog.created USING $orc2 AS SELECT id, data FROM source") - spark.sql(s"REPLACE TABLE $catalog.replaced USING $orc2 AS SELECT id, data FROM source") + spark.sql( + s"CREATE OR REPLACE TABLE $catalog.replaced USING $orc2 AS SELECT id, data FROM source") val testCatalog = spark.catalog(catalog).asTableCatalog val createdTable = testCatalog.loadTable(Identifier.of(Array(), "created")) @@ -302,6 +303,15 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn } } + test("ReplaceTableAsSelect: REPLACE TABLE throws exception if table does not exist.") { + Seq("testcat", "testcat_atomic").foreach { catalog => + spark.sql(s"CREATE TABLE $catalog.created USING $orc2 AS SELECT id, data FROM source") + intercept[CannotReplaceMissingTableException] { + spark.sql(s"REPLACE TABLE $catalog.replaced USING $orc2 AS SELECT id, data FROM source") + } + } + } + test("CreateTableAsSelect: use v2 plan because provider is v2") { spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") From 96b6db66aadb0d1013a1b7986bb26cd9ed5fa5af Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 8 Jul 2019 14:35:13 -0700 Subject: [PATCH 15/22] Name more boolean parameters --- .../datasources/v2/DataSourceV2Strategy.scala | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index c928892a94e3..b487c8fd92eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -178,9 +178,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case ReplaceTable(catalog, ident, schema, parts, props, orCreate) => catalog match { case staging: StagingTableCatalog => - AtomicReplaceTableExec(staging, ident, schema, parts, props, orCreate) :: Nil + AtomicReplaceTableExec(staging, ident, schema, parts, props, orCreate = orCreate) :: Nil case _ => - ReplaceTableExec(catalog, ident, schema, parts, props, orCreate) :: Nil + ReplaceTableExec(catalog, ident, schema, parts, props, orCreate = orCreate) :: Nil } case ReplaceTableAsSelect(catalog, ident, parts, query, props, options, orCreate) => @@ -188,10 +188,22 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { catalog match { case staging: StagingTableCatalog => AtomicReplaceTableAsSelectExec( - staging, ident, parts, planLater(query), props, writeOptions, orCreate) :: Nil + staging, + ident, + parts, + planLater(query), + props, + writeOptions, + orCreate = orCreate) :: Nil case _ => ReplaceTableAsSelectExec( - catalog, ident, parts, planLater(query), props, writeOptions, orCreate) :: Nil + catalog, + ident, + parts, + planLater(query), + props, + writeOptions, + orCreate = orCreate) :: Nil } case AppendData(r: DataSourceV2Relation, query, _) => From 295ff75aa8aabb36c8962b86f887e2aa12c2de12 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 11 Jul 2019 13:24:53 -0700 Subject: [PATCH 16/22] Remove REPLACE...TEMPORARY --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 13 ++----------- 2 files changed, 3 insertions(+), 12 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 13c6e3aa7a8b..48b15144cec7 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 @@ -270,7 +270,7 @@ createTableHeader ; replaceTableHeader - : (CREATE OR)? REPLACE TEMPORARY? TABLE multipartIdentifier + : (CREATE OR)? REPLACE TABLE multipartIdentifier ; bucketSpec diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 648c9943ad56..bdcad9418708 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2122,9 +2122,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitReplaceTableHeader( ctx: ReplaceTableHeaderContext): TableHeader = withOrigin(ctx) { - val temporary = ctx.TEMPORARY != null val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText) - (multipartIdentifier, temporary, false, false) + (multipartIdentifier, false, false, false) } /** @@ -2317,7 +2316,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * }}} */ override def visitReplaceTable(ctx: ReplaceTableContext): LogicalPlan = withOrigin(ctx) { - val (table, temp, ifNotExists, external) = visitReplaceTableHeader(ctx.replaceTableHeader) + val (table, _, ifNotExists, external) = visitReplaceTableHeader(ctx.replaceTableHeader) if (external) { operationNotAllowed("REPLACE EXTERNAL TABLE ... USING", ctx) } @@ -2342,9 +2341,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val orCreate = ctx.replaceTableHeader().CREATE() != null Option(ctx.query).map(plan) match { - case Some(_) if temp => - operationNotAllowed("REPLACE TEMPORARY TABLE ... USING ... AS query", ctx) - case Some(_) if schema.isDefined => operationNotAllowed( "Schema may not be specified in a Replace Table As Select (RTAS) statement", @@ -2354,11 +2350,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ReplaceTableAsSelectStatement(table, query, partitioning, bucketSpec, properties, provider, options, location, comment, orCreate = orCreate) - case None if temp => - // REPLACE TEMPORARY TABLE ... USING ... is not supported by the catalyst parser. - // Use REPLACE TEMPORARY VIEW ... USING ... instead. - operationNotAllowed("REPLACE TEMPORARY TABLE IF NOT EXISTS", ctx) - case _ => ReplaceTableStatement(table, schema.getOrElse(new StructType), partitioning, bucketSpec, properties, provider, options, location, comment, orCreate = orCreate) From deaf25507e88a8f095fbfc596d7178aa95b8a429 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 11 Jul 2019 17:07:39 -0700 Subject: [PATCH 17/22] Address comments --- .../apache/spark/sql/catalog/v2/StagingTableCatalog.java | 1 - .../catalyst/plans/logical/basicLogicalOperators.scala | 9 ++++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java index 97cd6c30c51c..0e6e579b04e7 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java @@ -74,7 +74,6 @@ StagedTable stageCreate( StructType schema, Transform[] partitions, Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; - ; /** * Stage the replacement of a table, preparing it to be committed into the metastore when the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 71caf71eec14..32c6a696fe82 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -443,8 +443,10 @@ case class CreateTableAsSelect( /** * Replace a table with a v2 catalog. * - * If the table does not exist, it will be created. The persisted table will have no contents - * as a result of this operation. + * If the table does not exist, and orCreate is true, then it will be created. + * If the table does not exist, and orCreate is false, then an exception will be thrown. + * + * The persisted table will have no contents as a result of this operation. */ case class ReplaceTable( catalog: TableCatalog, @@ -457,7 +459,8 @@ case class ReplaceTable( /** * Replaces a table from a select query with a v2 catalog. * - * If the table does not already exist, it will be created. + * If the table does not exist, and orCreate is true, then it will be created. + * If the table does not exist, and orCreate is false, then an exception will be thrown. */ case class ReplaceTableAsSelect( catalog: TableCatalog, From 581dba2822a8c1d23f9da653e8c26c84b18cfa3e Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 16 Jul 2019 15:29:08 -0700 Subject: [PATCH 18/22] Address comments --- .../sql/execution/datasources/v2/CreateTableExec.scala | 2 +- .../spark/sql/sources/v2/TestInMemoryTableCatalog.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala index b6f741954787..f35758bf08c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 136cb5a08185..3d2182810e20 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -209,9 +209,9 @@ object TestInMemoryTableCatalog { } def maybeSimulateFailedTableWrite(tableOptions: CaseInsensitiveStringMap): Unit = { - if (tableOptions.containsKey(TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION) - && tableOptions.get(TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION) - .equalsIgnoreCase("true")) { + val shouldSimulateFailedWrite = tableOptions + .getBoolean(TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION, false) + if (shouldSimulateFailedWrite) { throw new IllegalStateException("Manual write to table failure.") } } From be04476e968bd5cb5722c3b5a208b8430d78b1b9 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 17 Jul 2019 18:05:44 -0700 Subject: [PATCH 19/22] Address comments --- .../sql/catalog/v2/StagingTableCatalog.java | 3 +-- .../sources/v2/TestInMemoryTableCatalog.scala | 21 ++++++++++++------- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java index 0e6e579b04e7..77f0bc40dd86 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java @@ -92,7 +92,6 @@ StagedTable stageCreate( * @param partitions transforms to use for partitioning data in the table * @param properties a string map of table properties * @return metadata for the new table - * @throws TableAlreadyExistsException If a table or view already exists for the identifier * @throws UnsupportedOperationException If a requested partition transform is not supported * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) */ @@ -100,5 +99,5 @@ StagedTable stageReplace( Identifier ident, StructType schema, Transform[] partitions, - Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; + Map properties) throws NoSuchNamespaceException; } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 3d2182810e20..d444e64a067a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -40,6 +40,7 @@ class TestInMemoryTableCatalog extends TableCatalog { protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() + protected val droppedTables: mutable.Set[Identifier] = mutable.Set[Identifier]() private var _name: Option[String] = None override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { @@ -103,7 +104,11 @@ class TestInMemoryTableCatalog extends TableCatalog { } } - override def dropTable(ident: Identifier): Boolean = Option(tables.remove(ident)).isDefined + override def dropTable(ident: Identifier): Boolean = { + val isRemoved = Option(tables.remove(ident)).isDefined + droppedTables += ident + isRemoved + } def clearTables(): Unit = { tables.clear() @@ -201,17 +206,15 @@ object TestInMemoryTableCatalog { val SIMULATE_FAILED_CREATE_PROPERTY = "spark.sql.test.simulateFailedCreate" def maybeSimulateFailedTableCreation(tableProperties: util.Map[String, String]): Unit = { - if (tableProperties.containsKey(TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY) - && tableProperties.get(TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY) - .equalsIgnoreCase("true")) { + if ("true".equalsIgnoreCase( + tableProperties.get(TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY))) { throw new IllegalStateException("Manual create table failure.") } } def maybeSimulateFailedTableWrite(tableOptions: CaseInsensitiveStringMap): Unit = { - val shouldSimulateFailedWrite = tableOptions - .getBoolean(TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION, false) - if (shouldSimulateFailedWrite) { + if (tableOptions.getBoolean( + TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION, false)) { throw new IllegalStateException("Manual write to table failure.") } } @@ -263,6 +266,10 @@ class TestStagingInMemoryCatalog extends StagedTable with SupportsWrite with SupportsRead { override def commitStagedChanges(): Unit = { + if (droppedTables.contains(ident)) { + throw new IllegalStateException( + s"Table $ident was already dropped; aborting this commit.") + } if (replaceIfExists) { tables.put(ident, delegateTable) } else { From 609eb9c9292446b04cb8831d897087c6d1008efe Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 18 Jul 2019 18:06:48 -0700 Subject: [PATCH 20/22] Add stageCreateOrReplace to indicate to the staging catalog the appropriate semantics. --- .../sql/catalog/v2/StagingTableCatalog.java | 44 +++++++- .../v2/WriteToDataSourceV2Exec.scala | 11 +- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 11 ++ .../sources/v2/TestInMemoryTableCatalog.scala | 101 ++++++++++++------ 4 files changed, 128 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java index 77f0bc40dd86..c5aee0a304ff 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java @@ -82,10 +82,15 @@ StagedTable stageCreate( * When the table is committed, the contents of any writes performed by the Spark planner are * committed along with the metadata about the table passed into this method's arguments. If the * table exists, the metadata and the contents of this table replace the metadata and contents of - * the existing table. If the table does not exist, it should be created in the metastore. If a - * concurrent process commits changes to the table's data or metadata in the metastore while the - * write is being performed but before the staged changes are committed, the catalog can decide - * whether to move forward with the table replacement anyways or abort the commit operation. + * the existing table. If a concurrent process commits changes to the table's data or metadata + * while the write is being performed but before the staged changes are committed, the catalog + * can decide whether to move forward with the table replacement anyways or abort the commit + * operation. + *

+ * If the table does not exist, committing the staged changes should fail. This differs from the + * semantics of {@link #stageCreateOrReplace(Identifier, StructType, Transform[], Map)}, which + * should create the table in the data source if the table does not exist at the time of + * committing the operation. * * @param ident a table identifier * @param schema the schema of the new table, as a struct type @@ -100,4 +105,35 @@ StagedTable stageReplace( StructType schema, Transform[] partitions, Map properties) throws NoSuchNamespaceException; + + /** + * Stage the creation or replacement of a table, preparing it to be committed into the metastore + * when the returned table's {@link StagedTable#commitStagedChanges()} is called. + *

+ * When the table is committed, the contents of any writes performed by the Spark planner are + * committed along with the metadata about the table passed into this method's arguments. If the + * table exists, the metadata and the contents of this table replace the metadata and contents of + * the existing table. If a concurrent process commits changes to the table's data or metadata + * while the write is being performed but before the staged changes are committed, the catalog + * can decide whether to move forward with the table replacement anyways or abort the commit + * operation. + *

+ * If the table does not exist when the changes are committed, the table should be created in the + * backing data source. This differs from the expected semantics of + * {@link #stageReplace(Identifier, StructType, Transform[], Map)}, which should fail when + * the staged changes are committed but the table doesn't exist at commit time. + * + * @param ident a table identifier + * @param schema the schema of the new table, as a struct type + * @param partitions transforms to use for partitioning data in the table + * @param properties a string map of table properties + * @return metadata for the new table + * @throws UnsupportedOperationException If a requested partition transform is not supported + * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + */ + StagedTable stageCreateOrReplace( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws NoSuchNamespaceException; } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index e26ee74b9baf..a108f99d8c4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -214,10 +214,15 @@ case class AtomicReplaceTableAsSelectExec( override protected def doExecute(): RDD[InternalRow] = { val stagedTable = if (catalog.tableExists(ident)) { - catalog.stageReplace( - ident, query.schema, partitioning.toArray, properties.asJava) + if (orCreate) { + catalog.stageCreateOrReplace( + ident, query.schema, partitioning.toArray, properties.asJava) + } else { + catalog.stageReplace( + ident, query.schema, partitioning.toArray, properties.asJava) + } } else if (orCreate) { - catalog.stageCreate( + catalog.stageCreateOrReplace( ident, query.schema, partitioning.toArray, properties.asJava) } else { throw new CannotReplaceMissingTableException(ident) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index 86ea2693fe73..c173bdb95370 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -315,6 +315,17 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn } } + test("ReplaceTableAsSelect: REPLACE TABLE throws exception if table is dropped before commit.") { + import TestInMemoryTableCatalog._ + spark.sql(s"CREATE TABLE testcat_atomic.created USING $orc2 AS SELECT id, data FROM source") + intercept[CannotReplaceMissingTableException] { + spark.sql(s"REPLACE TABLE testcat_atomic.replaced" + + s" USING $orc2" + + s" TBLPROPERTIES (`$SIMULATE_DROP_BEFORE_REPLACE_PROPERTY`=true)" + + s" AS SELECT id, data FROM source") + } + } + test("CreateTableAsSelect: use v2 plan and session catalog when provider is v2") { spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index d444e64a067a..bf6e42d153b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, StagingT import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.types.StructType @@ -204,6 +204,7 @@ class InMemoryTable( object TestInMemoryTableCatalog { val SIMULATE_FAILED_WRITE_OPTION = "spark.sql.test.simulateFailedWrite" val SIMULATE_FAILED_CREATE_PROPERTY = "spark.sql.test.simulateFailedCreate" + val SIMULATE_DROP_BEFORE_REPLACE_PROPERTY = "spark.sql.test.simulateDropBeforeReplace" def maybeSimulateFailedTableCreation(tableProperties: util.Map[String, String]): Unit = { if ("true".equalsIgnoreCase( @@ -222,13 +223,18 @@ object TestInMemoryTableCatalog { class TestStagingInMemoryCatalog extends TestInMemoryTableCatalog with StagingTableCatalog { + import CatalogV2Implicits.IdentifierHelper + import org.apache.spark.sql.sources.v2.TestInMemoryTableCatalog._ override def stageCreate( ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - newStagedTable(ident, schema, partitions, properties, replaceIfExists = false) + validateStagedTable(partitions, properties) + new TestStagedCreateTable( + ident, + new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) } override def stageReplace( @@ -236,51 +242,39 @@ class TestStagingInMemoryCatalog schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - newStagedTable(ident, schema, partitions, properties, replaceIfExists = true) + validateStagedTable(partitions, properties) + new TestStagedReplaceTable( + ident, + new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) } - private def newStagedTable( + override def stageCreateOrReplace( ident: Identifier, schema: StructType, partitions: Array[Transform], - properties: util.Map[String, String], - replaceIfExists: Boolean): StagedTable = { - import CatalogV2Implicits.IdentifierHelper + properties: util.Map[String, String]): StagedTable = { + validateStagedTable(partitions, properties) + new TestStagedCreateOrReplaceTable( + ident, + new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) + } + + private def validateStagedTable( + partitions: Array[Transform], + properties: util.Map[String, String]): Unit = { if (partitions.nonEmpty) { throw new UnsupportedOperationException( s"Catalog $name: Partitioned tables are not supported") } - TestInMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) - - new TestStagedTable( - ident, - new InMemoryTable(s"$name.${ident.quoted}", schema, properties), - replaceIfExists) + maybeSimulateFailedTableCreation(properties) } - private class TestStagedTable( + private abstract class TestStagedTable( ident: Identifier, - delegateTable: InMemoryTable, - replaceIfExists: Boolean) + delegateTable: InMemoryTable) extends StagedTable with SupportsWrite with SupportsRead { - override def commitStagedChanges(): Unit = { - if (droppedTables.contains(ident)) { - throw new IllegalStateException( - s"Table $ident was already dropped; aborting this commit.") - } - if (replaceIfExists) { - tables.put(ident, delegateTable) - } else { - val maybePreCommittedTable = tables.putIfAbsent(ident, delegateTable) - if (maybePreCommittedTable != null) { - throw new TableAlreadyExistsException( - s"Table with identifier $ident and name $name was already created.") - } - } - } - override def abortStagedChanges(): Unit = {} override def name(): String = delegateTable.name @@ -297,8 +291,51 @@ class TestStagingInMemoryCatalog delegateTable.newScanBuilder(options) } } + + private class TestStagedCreateTable( + ident: Identifier, + delegateTable: InMemoryTable) extends TestStagedTable(ident, delegateTable) { + + override def commitStagedChanges(): Unit = { + val maybePreCommittedTable = tables.putIfAbsent(ident, delegateTable) + if (maybePreCommittedTable != null) { + throw new TableAlreadyExistsException( + s"Table with identifier $ident and name $name was already created.") + } + } + } + + private class TestStagedReplaceTable( + ident: Identifier, + delegateTable: InMemoryTable) extends TestStagedTable(ident, delegateTable) { + + override def commitStagedChanges(): Unit = { + maybeSimulateDropBeforeCommit() + val maybePreCommittedTable = tables.replace(ident, delegateTable) + if (maybePreCommittedTable == null) { + throw new CannotReplaceMissingTableException(ident) + } + } + + private def maybeSimulateDropBeforeCommit(): Unit = { + if ("true".equalsIgnoreCase( + delegateTable.properties.get(SIMULATE_DROP_BEFORE_REPLACE_PROPERTY))) { + tables.remove(ident) + } + } + } + + private class TestStagedCreateOrReplaceTable( + ident: Identifier, + delegateTable: InMemoryTable) extends TestStagedTable(ident, delegateTable) { + + override def commitStagedChanges(): Unit = { + tables.put(ident, delegateTable) + } + } } + class BufferedRows extends WriterCommitMessage with InputPartition with Serializable { val rows = new mutable.ArrayBuffer[InternalRow]() } From 2f6e0b64525771fbbe95f13ae25113435b5131ab Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 18 Jul 2019 18:11:11 -0700 Subject: [PATCH 21/22] Revert droppedTables stuff --- .../spark/sql/sources/v2/TestInMemoryTableCatalog.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index bf6e42d153b7..95398082b580 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -40,7 +40,6 @@ class TestInMemoryTableCatalog extends TableCatalog { protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() - protected val droppedTables: mutable.Set[Identifier] = mutable.Set[Identifier]() private var _name: Option[String] = None override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { @@ -105,9 +104,7 @@ class TestInMemoryTableCatalog extends TableCatalog { } override def dropTable(ident: Identifier): Boolean = { - val isRemoved = Option(tables.remove(ident)).isDefined - droppedTables += ident - isRemoved + Option(tables.remove(ident)).isDefined } def clearTables(): Unit = { From 05a827df7094b07a492add875c6e649df52db41f Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 19 Jul 2019 12:00:38 -0700 Subject: [PATCH 22/22] Address comments --- .../sql/catalog/v2/StagingTableCatalog.java | 13 ++++--- .../CannotReplaceMissingTableException.scala | 5 +-- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../datasources/v2/ReplaceTableExec.scala | 35 +++++++++---------- .../v2/WriteToDataSourceV2Exec.scala | 20 +++++------ 5 files changed, 39 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java index c5aee0a304ff..fc055e91a6ac 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalog.v2.expressions.Transform; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; import org.apache.spark.sql.sources.v2.StagedTable; import org.apache.spark.sql.sources.v2.SupportsWrite; @@ -87,10 +88,11 @@ StagedTable stageCreate( * can decide whether to move forward with the table replacement anyways or abort the commit * operation. *

- * If the table does not exist, committing the staged changes should fail. This differs from the - * semantics of {@link #stageCreateOrReplace(Identifier, StructType, Transform[], Map)}, which - * should create the table in the data source if the table does not exist at the time of - * committing the operation. + * If the table does not exist, committing the staged changes should fail with + * {@link NoSuchTableException}. This differs from the semantics of + * {@link #stageCreateOrReplace(Identifier, StructType, Transform[], Map)}, which should create + * the table in the data source if the table does not exist at the time of committing the + * operation. * * @param ident a table identifier * @param schema the schema of the new table, as a struct type @@ -99,12 +101,13 @@ StagedTable stageCreate( * @return metadata for the new table * @throws UnsupportedOperationException If a requested partition transform is not supported * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + * @throws NoSuchTableException If the table does not exist */ StagedTable stageReplace( Identifier ident, StructType schema, Transform[] partitions, - Map properties) throws NoSuchNamespaceException; + Map properties) throws NoSuchNamespaceException, NoSuchTableException; /** * Stage the creation or replacement of a table, preparing it to be committed into the metastore diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala index 6c05b4f10f0c..3036f7c21093 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala @@ -22,7 +22,8 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog.v2.Identifier class CannotReplaceMissingTableException( - tableIdentifier: Identifier) + tableIdentifier: Identifier, + cause: Option[Throwable] = None) extends AnalysisException( s"Table $tableIdentifier cannot be replaced as it did not exist." + - s" Use CREATE OR REPLACE TABLE to create the table.") + s" Use CREATE OR REPLACE TABLE to create the table.", cause = cause) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index ae2436b46dc9..e599128f8310 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2308,7 +2308,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * * Expected format: * {{{ - * REPLACE TABLE [IF NOT EXISTS] [db_name.]table_name + * [CREATE OR] REPLACE TABLE [db_name.]table_name * USING table_provider * replace_table_clauses * [[AS] select_statement]; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index cc22aa36dd5d..35d86ee2abbb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -23,7 +23,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.sources.v2.StagedTable @@ -32,19 +32,19 @@ import org.apache.spark.util.Utils case class ReplaceTableExec( catalog: TableCatalog, - identifier: Identifier, + ident: Identifier, tableSchema: StructType, partitioning: Seq[Transform], tableProperties: Map[String, String], orCreate: Boolean) extends LeafExecNode { override protected def doExecute(): RDD[InternalRow] = { - if (catalog.tableExists(identifier)) { - catalog.dropTable(identifier) + if (catalog.tableExists(ident)) { + catalog.dropTable(ident) } else if (!orCreate) { - throw new CannotReplaceMissingTableException(identifier) + throw new CannotReplaceMissingTableException(ident) } - catalog.createTable(identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + catalog.createTable(ident, tableSchema, partitioning.toArray, tableProperties.asJava) sqlContext.sparkContext.parallelize(Seq.empty, 1) } @@ -60,18 +60,17 @@ case class AtomicReplaceTableExec( orCreate: Boolean) extends LeafExecNode { override protected def doExecute(): RDD[InternalRow] = { - val staged = if (catalog.tableExists(identifier)) { - catalog.stageReplace( - identifier, - tableSchema, - partitioning.toArray, - tableProperties.asJava) - } else if (orCreate) { - catalog.stageCreate( - identifier, - tableSchema, - partitioning.toArray, - tableProperties.asJava) + val staged = if (orCreate) { + catalog.stageCreateOrReplace( + identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + } else if (catalog.tableExists(identifier)) { + try { + catalog.stageReplace( + identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + } catch { + case e: NoSuchTableException => + throw new CannotReplaceMissingTableException(identifier, Some(e)) + } } else { throw new CannotReplaceMissingTableException(identifier) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index a108f99d8c4e..9f644de1929a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -29,7 +29,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} @@ -213,21 +213,21 @@ case class AtomicReplaceTableAsSelectExec( orCreate: Boolean) extends AtomicTableWriteExec { override protected def doExecute(): RDD[InternalRow] = { - val stagedTable = if (catalog.tableExists(ident)) { - if (orCreate) { - catalog.stageCreateOrReplace( - ident, query.schema, partitioning.toArray, properties.asJava) - } else { + val staged = if (orCreate) { + catalog.stageCreateOrReplace( + ident, query.schema, partitioning.toArray, properties.asJava) + } else if (catalog.tableExists(ident)) { + try { catalog.stageReplace( ident, query.schema, partitioning.toArray, properties.asJava) + } catch { + case e: NoSuchTableException => + throw new CannotReplaceMissingTableException(ident, Some(e)) } - } else if (orCreate) { - catalog.stageCreateOrReplace( - ident, query.schema, partitioning.toArray, properties.asJava) } else { throw new CannotReplaceMissingTableException(ident) } - writeToStagedTable(stagedTable, writeOptions, ident) + writeToStagedTable(staged, writeOptions, ident) } }