From d5b0d4ad5f276a9f9fd52154c782418b8fa80a28 Mon Sep 17 00:00:00 2001 From: John Zhuge Date: Wed, 29 May 2019 20:02:25 -0700 Subject: [PATCH] Review comments --- .../spark/sql/catalog/v2/IdentifierImpl.java | 17 +++++++++++++++++ .../spark/sql/catalyst/parser/AstBuilder.scala | 10 +++++----- .../plans/logical/sql/DropViewStatement.scala | 2 +- .../datasources/DataSourceResolution.scala | 9 +++++++-- .../execution/command/PlanResolutionSuite.scala | 13 +++++++------ .../sql/sources/v2/DataSourceV2SQLSuite.scala | 3 +-- 6 files changed, 38 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java index cd131432008a6..34f3882c9c412 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java @@ -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. @@ -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) { 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 7092868517f19..fa05efebf9c6d 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} +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._ @@ -2196,20 +2196,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Create a [[sql.DropTableStatement]] command. + * Create a [[DropTableStatement]] command. */ override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { - sql.DropTableStatement( + DropTableStatement( visitMultipartIdentifier(ctx.multipartIdentifier()), ctx.EXISTS != null, ctx.PURGE != null) } /** - * Create a [[sql.DropViewStatement]] command. + * Create a [[DropViewStatement]] command. */ override def visitDropView(ctx: DropViewContext): AnyRef = withOrigin(ctx) { - sql.DropViewStatement( + DropViewStatement( visitMultipartIdentifier(ctx.multipartIdentifier()), ctx.EXISTS != null) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala index 7a12db000d3ed..523158788e834 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan * A DROP VIEW statement, as parsed from SQL. */ case class DropViewStatement( - tableName: Seq[String], + viewName: Seq[String], ifExists: Boolean) extends ParsedStatement { override def output: Seq[Attribute] = Seq.empty 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 96e077931e665..58b9276869ab2 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 @@ -89,10 +89,15 @@ case class DataSourceResolution( DropTable(catalog.asTableCatalog, ident, ifExists) case DropTableStatement(AsTableIdentifier(tableName), ifExists, purge) => - DropTableCommand(tableName, ifExists, false, 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, true, false) + DropTableCommand(tableName, ifExists, isView = true, purge = false) } object V1WriteProvider { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index d62ae616e5394..06f7332086372 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.command import java.net.URI +import java.util.Locale import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, TableCatalog, TestTableCatalog} @@ -462,7 +463,7 @@ class PlanResolutionSuite extends AnalysisTest { DropTableCommand(tableIdent2, ifExists = true, isView = false, purge = true)) } - test("drop table v2") { + test("drop table in v2 catalog") { val tableName1 = "testcat.db.tab" val tableIdent1 = Identifier.of(Array("db"), "tab") val tableName2 = "testcat.tab" @@ -494,10 +495,10 @@ class PlanResolutionSuite extends AnalysisTest { DropTableCommand(viewIdent2, ifExists = true, isView = true, purge = false)) } - test("drop view v2") { - assertAnalysisError( - parseAndResolve("DROP VIEW testcat.db.view"), - Seq("unresolved operator 'DropViewStatement"), - caseSensitive = false) + test("drop view in v2 catalog") { + intercept[AnalysisException] { + parseAndResolve("DROP VIEW testcat.db.view") + }.getMessage.toLowerCase(Locale.ROOT).contains( + "view support in catalog has not been implemented") } } 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 2cccd62edf3a0..5b9071b59b9b0 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 @@ -28,8 +28,7 @@ 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} -class DataSourceV2SQLSuite - extends QueryTest with SharedSQLContext with BeforeAndAfter { +class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._