Skip to content

Commit

Permalink
[SPARK-27813][SQL] DataSourceV2: Add DropTable logical operation
Browse files Browse the repository at this point in the history
## What changes were proposed in this pull request?

Support DROP TABLE from V2 catalogs.
Move DROP TABLE into catalyst.
Move parsing tests for DROP TABLE/VIEW to PlanResolutionSuite to validate existing behavior.
Add new tests fo catalyst parser suite.
Separate DROP VIEW into different code path from DROP TABLE.
Move DROP VIEW into catalyst as a new operator.
Add a meaningful exception to indicate view is not currently supported in v2 catalog.

## How was this patch tested?

New unit tests.
Existing unit tests in catalyst and sql core.

Closes apache#24686 from jzhuge/SPARK-27813-pr.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
jzhuge authored and mccheah committed Jun 6, 2019
1 parent 6244b77 commit 0db2aa0
Show file tree
Hide file tree
Showing 14 changed files with 294 additions and 81 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -136,8 +136,8 @@ statement
DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions
| ALTER TABLE tableIdentifier partitionSpec? SET locationSpec #setTableLocation
| ALTER TABLE tableIdentifier RECOVER PARTITIONS #recoverPartitions
| DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable
| DROP VIEW (IF EXISTS)? tableIdentifier #dropTable
| DROP TABLE (IF EXISTS)? multipartIdentifier PURGE? #dropTable
| DROP VIEW (IF EXISTS)? multipartIdentifier #dropView
| CREATE (OR REPLACE)? (GLOBAL? TEMPORARY)?
VIEW (IF NOT EXISTS)? tableIdentifier
identifierCommentList? (COMMENT STRING)?
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@

import java.util.Arrays;
import java.util.Objects;
import java.util.stream.Collectors;
import java.util.stream.Stream;

/**
* An {@link Identifier} implementation.
Expand Down Expand Up @@ -49,6 +51,21 @@ public String name() {
return name;
}

private String escapeQuote(String part) {
if (part.contains("`")) {
return part.replace("`", "``");
} else {
return part;
}
}

@Override
public String toString() {
return Stream.concat(Stream.of(namespace), Stream.of(name))
.map(part -> '`' + escapeQuote(part) + '`')
.collect(Collectors.joining("."));
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,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}
import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement}
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 @@ -2151,4 +2151,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}
}

/**
* Create a [[DropTableStatement]] command.
*/
override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
DropTableStatement(
visitMultipartIdentifier(ctx.multipartIdentifier()),
ctx.EXISTS != null,
ctx.PURGE != null)
}

/**
* Create a [[DropViewStatement]] command.
*/
override def visitDropView(ctx: DropViewContext): AnyRef = withOrigin(ctx) {
DropViewStatement(
visitMultipartIdentifier(ctx.multipartIdentifier()),
ctx.EXISTS != null)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -484,6 +484,14 @@ object OverwritePartitionsDynamic {
}
}

/**
* Drop a table.
*/
case class DropTable(
catalog: TableCatalog,
ident: Identifier,
ifExists: Boolean) extends Command


/**
* Insert some data into a table. Note that this plan is unresolved and has to be replaced by the
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan

/**
* A DROP TABLE statement, as parsed from SQL.
*/
case class DropTableStatement(
tableName: Seq[String],
ifExists: Boolean,
purge: Boolean) extends ParsedStatement {

override def output: Seq[Attribute] = Seq.empty

override def children: Seq[LogicalPlan] = Seq.empty
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan

/**
* A DROP VIEW statement, as parsed from SQL.
*/
case class DropViewStatement(
viewName: Seq[String],
ifExists: Boolean) extends ParsedStatement {

override def output: Seq[Attribute] = Seq.empty

override def children: Seq[LogicalPlan] = Seq.empty
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,8 @@ 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 org.apache.spark.sql.catalyst.analysis.AnalysisTest
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement}
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.types.{IntegerType, StringType, StructType, TimestampType}
import org.apache.spark.unsafe.types.UTF8String

Expand All @@ -34,6 +35,10 @@ class DDLParserSuite extends AnalysisTest {
}
}

private def parseCompare(sql: String, expected: LogicalPlan): Unit = {
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"

Expand Down Expand Up @@ -362,4 +367,31 @@ class DDLParserSuite extends AnalysisTest {
}
}
}

test("drop table") {
parseCompare("DROP TABLE testcat.ns1.ns2.tbl",
DropTableStatement(Seq("testcat", "ns1", "ns2", "tbl"), ifExists = false, purge = false))
parseCompare(s"DROP TABLE db.tab",
DropTableStatement(Seq("db", "tab"), ifExists = false, purge = false))
parseCompare(s"DROP TABLE IF EXISTS db.tab",
DropTableStatement(Seq("db", "tab"), ifExists = true, purge = false))
parseCompare(s"DROP TABLE tab",
DropTableStatement(Seq("tab"), ifExists = false, purge = false))
parseCompare(s"DROP TABLE IF EXISTS tab",
DropTableStatement(Seq("tab"), ifExists = true, purge = false))
parseCompare(s"DROP TABLE tab PURGE",
DropTableStatement(Seq("tab"), ifExists = false, purge = true))
parseCompare(s"DROP TABLE IF EXISTS tab PURGE",
DropTableStatement(Seq("tab"), ifExists = true, purge = true))
}

test("drop view") {
parseCompare(s"DROP VIEW testcat.db.view",
DropViewStatement(Seq("testcat", "db", "view"), ifExists = false))
parseCompare(s"DROP VIEW db.view", DropViewStatement(Seq("db", "view"), ifExists = false))
parseCompare(s"DROP VIEW IF EXISTS db.view",
DropViewStatement(Seq("db", "view"), ifExists = true))
parseCompare(s"DROP VIEW view", DropViewStatement(Seq("view"), ifExists = false))
parseCompare(s"DROP VIEW IF EXISTS view", DropViewStatement(Seq("view"), ifExists = true))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -646,17 +646,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
ctx.TEMPORARY != null)
}

/**
* Create a [[DropTableCommand]] command.
*/
override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
DropTableCommand(
visitTableIdentifier(ctx.tableIdentifier),
ctx.EXISTS != null,
ctx.VIEW != null,
ctx.PURGE != null)
}

/**
* Create a [[AlterTableRenameCommand]] command.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,10 @@ 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, LogicalPlan}
import org.apache.spark.sql.catalyst.plans.logical.sql.{CreateTableAsSelectStatement, CreateTableStatement}
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.rules.Rule
import org.apache.spark.sql.execution.command.DropTableCommand
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.v2.TableProvider
import org.apache.spark.sql.types.StructType
Expand Down Expand Up @@ -81,6 +82,20 @@ case class DataSourceResolution(
s"No catalog specified for table ${identifier.quoted} and no default catalog is set"))
.asTableCatalog
convertCTAS(catalog, identifier, create)

case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) =>
DropTable(catalog.asTableCatalog, ident, ifExists)

case DropTableStatement(AsTableIdentifier(tableName), ifExists, purge) =>
DropTableCommand(tableName, ifExists, isView = false, purge)

case DropViewStatement(CatalogObjectIdentifier(Some(catalog), ident), _) =>
throw new AnalysisException(
s"Can not specify catalog `${catalog.name}` for view $ident " +
s"because view support in catalog has not been implemented yet")

case DropViewStatement(AsTableIdentifier(tableName), ifExists) =>
DropTableCommand(tableName, ifExists, isView = true, purge = false)
}

object V1WriteProvider {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition}
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition}
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}
Expand Down Expand Up @@ -187,6 +187,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
Nil
}

case DropTable(catalog, ident, ifExists) =>
DropTableExec(catalog, ident, ifExists) :: Nil

case _ => Nil
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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 org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.execution.LeafExecNode

/**
* Physical plan node for dropping a table.
*/
case class DropTableExec(catalog: TableCatalog, ident: Identifier, ifExists: Boolean)
extends LeafExecNode {

override def doExecute(): RDD[InternalRow] = {
if (catalog.tableExists(ident)) {
catalog.dropTable(ident)
} else if (!ifExists) {
throw new NoSuchTableException(ident)
}

sqlContext.sparkContext.parallelize(Seq.empty, 1)
}

override def output: Seq[Attribute] = Seq.empty
}
Loading

0 comments on commit 0db2aa0

Please sign in to comment.