[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-11-04 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r517786526



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##
@@ -228,8 +228,11 @@ class DataSourceV2Strategy(session: SparkSession) extends 
Strategy with Predicat
 case DescribeColumn(_: ResolvedTable, _, _) =>
   throw new AnalysisException("Describing columns is not supported for v2 
tables.")
 
-case DropTable(catalog, ident, ifExists) =>
-  DropTableExec(catalog, ident, ifExists) :: Nil
+case DropTable(r: ResolvedTable, ifExists, _) =>

Review comment:
   Silently ignore the `purge` option as the default behavior looks risky. 
How about making the new variant of `dropTable` fail by default if 
`purge==true`?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-11-03 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r517072152



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##
@@ -228,8 +228,11 @@ class DataSourceV2Strategy(session: SparkSession) extends 
Strategy with Predicat
 case DescribeColumn(_: ResolvedTable, _, _) =>
   throw new AnalysisException("Describing columns is not supported for v2 
tables.")
 
-case DropTable(catalog, ident, ifExists) =>
-  DropTableExec(catalog, ident, ifExists) :: Nil
+case DropTable(r: ResolvedTable, ifExists, _) =>

Review comment:
   How useful is the `purge` option? If it's rarely used, we can just fail 
v2 DROP TABLE if `purge` is specified.
   
   cc @rdblue @brkyvz 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-27 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r513193419



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
##
@@ -367,9 +367,17 @@ class ResolveSessionCatalog(
   orCreate = c.orCreate)
   }
 
+case DropTable(
+r @ ResolvedTable(_, _, _: V1Table), ifExists, purge) if 
isSessionCatalog(r.catalog) =>
+  DropTableCommand(r.identifier.asTableIdentifier, ifExists, isView = 
false, purge = purge)
+
 // v1 DROP TABLE supports temp view.
-case DropTableStatement(TempViewOrV1Table(name), ifExists, purge) =>
-  DropTableCommand(name.asTableIdentifier, ifExists, isView = false, purge 
= purge)
+case DropTable(r: ResolvedView, ifExists, purge) =>
+  if (!r.isTemp) {

Review comment:
   Currently, it's duplicated with the check in `DropTableCommand`. I think 
it's OK as we want to move to v2 commands eventually.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-27 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r512772861



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
##
@@ -368,9 +368,13 @@ class ResolveSessionCatalog(
   orCreate = c.orCreate)
   }
 
+case DropTable(
+r @ ResolvedTable(_, _, _: V1Table), ifExists, purge) if 
isSessionCatalog(r.catalog) =>
+  DropTableCommand(r.identifier.asTableIdentifier, ifExists, isView = 
false, purge = purge)
+
 // v1 DROP TABLE supports temp view.
-case DropTableStatement(TempViewOrV1Table(name), ifExists, purge) =>
-  DropTableCommand(name.asTableIdentifier, ifExists, isView = false, purge 
= purge)
+case DropTable(r: ResolvedView, ifExists, purge) =>

Review comment:
   just `assert(r.isTemp)`





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-26 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r511821763



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
##
@@ -368,9 +368,13 @@ class ResolveSessionCatalog(
   orCreate = c.orCreate)
   }
 
+case DropTable(
+r @ ResolvedTable(_, _, _: V1Table), ifExists, purge) if 
isSessionCatalog(r.catalog) =>
+  DropTableCommand(r.identifier.asTableIdentifier, ifExists, isView = 
false, purge = purge)
+
 // v1 DROP TABLE supports temp view.
-case DropTableStatement(TempViewOrV1Table(name), ifExists, purge) =>
-  DropTableCommand(name.asTableIdentifier, ifExists, isView = false, purge 
= purge)
+case DropTable(r: ResolvedView, ifExists, purge) =>

Review comment:
   nit: shall we add an assert here to make sure it's not temp view?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-22 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r510594110



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##
@@ -228,8 +228,11 @@ class DataSourceV2Strategy(session: SparkSession) extends 
Strategy with Predicat
 case DescribeColumn(_: ResolvedTable, _, _) =>
   throw new AnalysisException("Describing columns is not supported for v2 
tables.")
 
-case DropTable(catalog, ident, ifExists) =>
-  DropTableExec(catalog, ident, ifExists) :: Nil
+case DropTable(r: ResolvedTable, ifExists, _) =>

Review comment:
   not related to this PR, but we should think about how to handle the 
`purge` flag in v2 API.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-22 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r510593591



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala
##
@@ -74,8 +82,11 @@ private[connector] trait TestV2SessionCatalogBase[T <: 
Table] extends Delegating
   }
 
   def clearTables(): Unit = {
-assert(!tables.isEmpty, "Tables were empty, maybe didn't use the session 
catalog code path?")
+assert(
+  tableCreated.get,

Review comment:
   Why doesn't `!tables.isEmpty` work?

##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala
##
@@ -74,8 +82,11 @@ private[connector] trait TestV2SessionCatalogBase[T <: 
Table] extends Delegating
   }
 
   def clearTables(): Unit = {
-assert(!tables.isEmpty, "Tables were empty, maybe didn't use the session 
catalog code path?")
+assert(
+  tableCreated.get,

Review comment:
   Why doesn't `!tables.isEmpty` work?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-22 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r510592895



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##
@@ -228,8 +228,11 @@ class DataSourceV2Strategy(session: SparkSession) extends 
Strategy with Predicat
 case DescribeColumn(_: ResolvedTable, _, _) =>
   throw new AnalysisException("Describing columns is not supported for v2 
tables.")
 
-case DropTable(catalog, ident, ifExists) =>
-  DropTableExec(catalog, ident, ifExists) :: Nil
+case DropTable(r: ResolvedTable, ifExists, _) =>
+  DropTableExec(r.catalog, r.identifier, ifExists) :: Nil
+
+case NoopDropTable(multipartIdentifier) =>
+  NoopDropTableExec(multipartIdentifier) :: Nil

Review comment:
   how about `LocalTableScanExec(Nil, Nil)`? It's only for EXPLAIN, so a 
logical plan is good enough. We don't need to create a real physical plan for 
it.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-22 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r510587663



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala
##
@@ -81,7 +81,7 @@ case class ResolvedTable(catalog: TableCatalog, identifier: 
Identifier, table: T
  */
 // TODO: create a generic representation for temp view, v1 view and v2 view, 
after we add view
 //   support to v2 catalog. For now we only need the identifier to 
fallback to v1 command.
-case class ResolvedView(identifier: Identifier) extends LeafNode {
+case class ResolvedView(identifier: Identifier, isTempView: Boolean) extends 
LeafNode {

Review comment:
   nit: the class name is `ResolvedView`, so the flag can be named as 
`isTemp`, as we are already in the context of view.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-22 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r510587530



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveNoopDropTable.scala
##
@@ -0,0 +1,40 @@
+/*
+ * 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.catalyst.plans.logical.{DropTable, LogicalPlan, 
NoopDropTable}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * A rule for handling [[DropTable]] logical plan when the table or temp view 
is not resolved.
+ * If "ifExists" flag is set to true, the plan is resolved to 
[[NoopDropTable]],
+ * which is a no-op command. If the flag is set to false, the rule throws 
[[NoSuchTableException]].
+ */
+object ResolveNoopDropTable extends Rule[LogicalPlan] {
+  import 
org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
+
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+case DropTable(u: UnresolvedTableOrView, ifExists, _) =>
+  if (ifExists) {
+NoopDropTable(u.multipartIdentifier)
+  } else {
+throw new NoSuchTableException(

Review comment:
   We can leave it to the `CheckAnalysis`





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #30079: [SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

2020-10-22 Thread GitBox


cloud-fan commented on a change in pull request #30079:
URL: https://github.com/apache/spark/pull/30079#discussion_r509947710



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala
##
@@ -45,12 +45,25 @@ case class UnresolvedTable(multipartIdentifier: 
Seq[String]) extends LeafNode {
 /**
  * Holds the name of a table or view that has yet to be looked up in a 
catalog. It will
  * be resolved to [[ResolvedTable]] or [[ResolvedView]] during analysis.
+ *
+ * If 'isResolutionRequired' is set to false and the name cannot be resolved 
to a table or view,
+ * [[UnresolvedTableOrView]] will be converted to [[NotFoundTableOrView]].
  */
-case class UnresolvedTableOrView(multipartIdentifier: Seq[String]) extends 
LeafNode {
+case class UnresolvedTableOrView(
+multipartIdentifier: Seq[String],
+isResolutionRequired: Boolean = true) extends LeafNode {
   override lazy val resolved: Boolean = false
   override def output: Seq[Attribute] = Nil
 }
 
+/**
+ * Holds the name of a table or view that has been looked up in a catalog, but 
not found.
+ * This is a "resolved" logical.
+ */
+case class NotFoundTableOrView(multipartIdentifier: Seq[String]) extends 
LeafNode {

Review comment:
   If this is specific to DROP TABLE, let's avoid changing the framework. 
How about we add a rule in the `Post-Hoc Resolution` batch in the analyzer:
   ```
   object ResolveNoopDropTable extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan) = plan.transform {
case DropTable(_: UnresolvedTableOrView, _, true) => NoopDropTable
 }
   }
   ```
   where `NoopDropTable` is a new command that does nothing but only keeping 
the table name for EXPLAIN.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org