Skip to content

Commit

Permalink
[SPARK-27724][SQL] Implement REPLACE TABLE and REPLACE TABLE AS SELEC…
Browse files Browse the repository at this point in the history
…T with V2

## What changes were proposed in this pull request?

Implements the `REPLACE TABLE` and `REPLACE TABLE AS SELECT` logical plans. `REPLACE TABLE` is now a valid operation in spark-sql provided that the tables being modified are managed by V2 catalogs.

This also introduces an atomic mix-in that table catalogs can choose to implement. Table catalogs can now implement `TransactionalTableCatalog`. The semantics of this API are that table creation and replacement can be "staged" and then "committed".

On the execution of `REPLACE TABLE AS SELECT`, `REPLACE TABLE`, and `CREATE TABLE AS SELECT`, if the catalog implements transactional operations, the physical plan will use said functionality. Otherwise, these operations fall back on non-atomic variants. For `REPLACE TABLE` in particular, the usage of non-atomic operations can unfortunately lead to inconsistent state.

## How was this patch tested?

Unit tests - multiple additions to `DataSourceV2SQLSuite`.

Closes apache#24798 from mccheah/spark-27724.

Authored-by: mcheah <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
  • Loading branch information
mccheah authored and cloud-fan committed Jul 22, 2019
1 parent a783690 commit 7ed0088
Show file tree
Hide file tree
Showing 14 changed files with 1,402 additions and 258 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,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 multipartIdentifier
Expand Down Expand Up @@ -261,6 +269,10 @@ createTableHeader
: CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? multipartIdentifier
;

replaceTableHeader
: (CREATE OR)? REPLACE TABLE multipartIdentifier
;

bucketSpec
: CLUSTERED BY identifierList
(SORTED BY orderedIdentifierList)?
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
/*
* 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.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;
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.
* <p>
* 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.
* <p>
* 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.
* <p>
* 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<String, String> 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.
* <p>
* 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.
* <p>
* 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
* @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)
* @throws NoSuchTableException If the table does not exist
*/
StagedTable stageReplace(
Identifier ident,
StructType schema,
Transform[] partitions,
Map<String, String> properties) throws NoSuchNamespaceException, NoSuchTableException;

/**
* 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.
* <p>
* 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.
* <p>
* 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<String, String> properties) throws NoSuchNamespaceException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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;

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.
* <p>
* 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();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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,
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.", cause = cause)
Original file line number Diff line number Diff line change
Expand Up @@ -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.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, 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._
Expand Down Expand Up @@ -2127,6 +2127,15 @@ 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 multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText)
(multipartIdentifier, false, false, false)
}

/**
* Parse a qualified name to a multipart name.
*/
Expand Down Expand Up @@ -2294,6 +2303,69 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}
}

/**
* Replace a table, returning a [[ReplaceTableStatement]] logical plan.
*
* Expected format:
* {{{
* [CREATE OR] REPLACE TABLE [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, _, 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)
val orCreate = ctx.replaceTableHeader().CREATE() != null

Option(ctx.query).map(plan) match {
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, orCreate = orCreate)

case _ =>
ReplaceTableStatement(table, schema.getOrElse(new StructType), partitioning,
bucketSpec, properties, provider, options, location, comment, orCreate = orCreate)
}
}

/**
* Create a [[DropTableStatement]] command.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -441,6 +441,47 @@ case class CreateTableAsSelect(
}
}

/**
* Replace a table with a v2 catalog.
*
* 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,
tableName: Identifier,
tableSchema: StructType,
partitioning: Seq[Transform],
properties: Map[String, String],
orCreate: Boolean) extends Command

/**
* Replaces a table from a select query with a v2 catalog.
*
* 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,
tableName: Identifier,
partitioning: Seq[Transform],
query: LogicalPlan,
properties: Map[String, String],
writeOptions: Map[String, String],
orCreate: Boolean) 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.
*/
Expand Down
Loading

0 comments on commit 7ed0088

Please sign in to comment.