[GitHub] [spark] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-29 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r549937965



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
##
@@ -764,13 +765,13 @@ case class DescribeColumnCommand(
 val colName = UnresolvedAttribute(colNameParts).name
 val field = {
   relation.resolve(colNameParts, resolver).getOrElse {
-throw new AnalysisException(s"Column $colName does not exist")
+throw QueryCompilationErrors.columnDoesNotExistError(colName)
   }
 }
 if (!field.isInstanceOf[Attribute]) {
   // If the field is not an attribute after `resolve`, then it's a nested 
field.
-  throw new AnalysisException(
-s"DESC TABLE COLUMN command does not support nested data types: 
$colName")
+  throw QueryCompilationErrors.commandNotSupportNestedColumnError(
+"DESC TABLE COLUMN", toPrettySQL(field, removeAlias = true))

Review comment:
   I think this can still be hit for views.





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-29 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r549907312



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
##
@@ -154,13 +154,54 @@ class DataSourceV2SQLSuite
   Array("Table Properties", "[bar=baz]", "")))
   }
 
-  test("Describe column is not supported for v2 catalog") {
-withTable("testcat.tbl") {
-  spark.sql("CREATE TABLE testcat.tbl (id bigint) USING foo")
-  val ex = intercept[AnalysisException] {
-spark.sql("DESCRIBE testcat.tbl id")
+  test("Describe column for v2 catalog") {
+val t = "testcat.tbl"
+withTable(t) {
+  sql(s"CREATE TABLE $t (id bigint, data string COMMENT 'hello') USING 
foo")
+  val df1 = sql(s"DESCRIBE $t id")
+  assert(df1.schema.map(field => (field.name, field.dataType))
+=== Seq(("info_name", StringType), ("info_value", StringType)))
+  assert(df1.collect === Seq(
+Row("col_name", "id"),
+Row("data_type", "bigint"),
+Row("comment", "NULL")))
+  val df2 = sql(s"DESCRIBE $t data")
+  assert(df2.schema.map(field => (field.name, field.dataType))
+=== Seq(("info_name", StringType), ("info_value", StringType)))
+  assert(df2.collect === Seq(
+Row("col_name", "data"),
+Row("data_type", "string"),
+Row("comment", "hello")))
+
+  assertAnalysisError(
+s"DESCRIBE $t invalid_col",
+"cannot resolve '`invalid_col`' given input columns: 
[testcat.tbl.data, testcat.tbl.id]")

Review comment:
   The error message is different for v1 / v2 tables when the column does 
not exist.
   v1: `Column invalid_col does not exist`
   v2: ```cannot resolve '`invalid_col`' given input columns: 
[testcat.tbl.data, testcat.tbl.id]```
   `CheckAnalysis` handles `UnresolvedAttribute` automatically for v2. Should 
we make this consistent (i.e., make v2 emit messages like v1)?





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-29 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r549906670



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
##
@@ -235,8 +236,17 @@ class ResolveSessionCatalog(
 case DescribeRelation(ResolvedV1TableOrViewIdentifier(ident), 
partitionSpec, isExtended) =>
   DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended)
 
-case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), colNameParts, 
isExtended) =>
-  DescribeColumnCommand(ident.asTableIdentifier, colNameParts, isExtended)
+case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), column, 
isExtended) =>
+  column match {
+case u: UnresolvedAttribute =>
+  // For views, the column will not be resolved by `ResolveReferences` 
because
+  // `ResolvedView` stores only the identifier.
+  DescribeColumnCommand(ident.asTableIdentifier, u.nameParts, 
isExtended)

Review comment:
   The comment was confusing since `DescribeColumnCommand` resolves the 
column again. I updated to separate view and table matching to make the 
intention clear. Thanks.





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-29 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r549906401



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
##
@@ -235,8 +236,17 @@ class ResolveSessionCatalog(
 case DescribeRelation(ResolvedV1TableOrViewIdentifier(ident), 
partitionSpec, isExtended) =>
   DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended)
 
-case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), colNameParts, 
isExtended) =>
-  DescribeColumnCommand(ident.asTableIdentifier, colNameParts, isExtended)
+case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), column, 
isExtended) =>
+  column match {
+case u: UnresolvedAttribute =>
+  // For views, the column will not be resolved by `ResolveReferences` 
because
+  // `ResolvedView` stores only the identifier.
+  DescribeColumnCommand(ident.asTableIdentifier, u.nameParts, 
isExtended)
+case a: Attribute =>
+  DescribeColumnCommand(ident.asTableIdentifier, a.qualifier :+ 
a.name, isExtended)
+case nested =>
+  throw 
QueryCompilationErrors.commandNotSupportNestedColumnError("DESC TABLE COLUMN")

Review comment:
   That seems to work pretty well. Thanks!





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-28 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r549600597



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
##
@@ -235,8 +236,17 @@ class ResolveSessionCatalog(
 case DescribeRelation(ResolvedV1TableOrViewIdentifier(ident), 
partitionSpec, isExtended) =>
   DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended)
 
-case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), colNameParts, 
isExtended) =>
-  DescribeColumnCommand(ident.asTableIdentifier, colNameParts, isExtended)
+case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), column, 
isExtended) =>
+  column match {
+case u: UnresolvedAttribute =>
+  // For views, the column will not be resolved by `ResolveReferences` 
because
+  // `ResolvedView` stores only the identifier.
+  DescribeColumnCommand(ident.asTableIdentifier, u.nameParts, 
isExtended)
+case a: Attribute =>
+  DescribeColumnCommand(ident.asTableIdentifier, a.qualifier :+ 
a.name, isExtended)
+case nested =>
+  throw 
QueryCompilationErrors.commandNotSupportNestedColumnError("DESC TABLE COLUMN")

Review comment:
   For `DESC desc_complex_col_table col.x`,
   It will be:
   ```
   DESC TABLE COLUMN command does not support nested data types: col.x
   ```
   vs.
   ```
   DESC TABLE COLUMN does not support nested column: 
spark_catalog.default.desc_complex_col_table.`col`.`x` AS `x`
   ```





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-28 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r549576786



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala
##
@@ -97,9 +98,26 @@ case class ResolvedNamespace(catalog: CatalogPlugin, 
namespace: Seq[String])
 /**
  * A plan containing resolved table.
  */
-case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: 
Table)
+case class ResolvedTable(
+catalog: TableCatalog,
+identifier: Identifier,
+table: Table,
+outputAttributes: Seq[Attribute])
   extends LeafNode {
-  override def output: Seq[Attribute] = Nil
+  override def output: Seq[Attribute] = {
+val qualifier = catalog.name +: identifier.namespace :+ identifier.name
+outputAttributes.map(_.withQualifier(qualifier))

Review comment:
   Or we can wrap this with `SubqueryAlias` similar to how 
`DataSourceV2Relation` is wrapped, but we need to update everywhere 
`ResolvedTable` is matched.





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-28 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r549529945



##
File path: sql/core/src/test/resources/sql-tests/results/describe.sql.out
##
@@ -561,7 +561,7 @@ struct
 -- !query output
 == Physical Plan ==
 Execute DescribeColumnCommand
-   +- DescribeColumnCommand `default`.`t`, [b], false
+   +- DescribeColumnCommand `default`.`t`, [spark_catalog, default, t, b], 
false

Review comment:
   Now that attribute is resolved, a fully qualified name is passed to 
`DescribeColumnCommand`





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-28 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r549528750



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
##
@@ -235,8 +236,17 @@ class ResolveSessionCatalog(
 case DescribeRelation(ResolvedV1TableOrViewIdentifier(ident), 
partitionSpec, isExtended) =>
   DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended)
 
-case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), colNameParts, 
isExtended) =>
-  DescribeColumnCommand(ident.asTableIdentifier, colNameParts, isExtended)
+case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), column, 
isExtended) =>
+  column match {
+case u: UnresolvedAttribute =>
+  // For views, the column will not be resolved by `ResolveReferences` 
because
+  // `ResolvedView` stores only the identifier.
+  DescribeColumnCommand(ident.asTableIdentifier, u.nameParts, 
isExtended)
+case a: Attribute =>
+  DescribeColumnCommand(ident.asTableIdentifier, a.qualifier :+ 
a.name, isExtended)
+case nested =>
+  throw 
QueryCompilationErrors.commandNotSupportNestedColumnError("DESC TABLE COLUMN")

Review comment:
   One disadvantage of this approach is that the exception message for view 
will be different when nested column is specified; it will have the original 
name parts: 
https://github.com/apache/spark/blob/0617dfce7beb34662ab30a607721e9b46e65c21e/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala#L773
   
   We can do one of the following:
   * Make the exception message the same even for views by dropping column name 
in `DescribeColumnCommand`
   * Store the original column name in `DescribeColumn` (and there will be no 
matching logic for column in `ResolveSessionCatalog`, but seems duplicated 
because we have `UnresolvedAttribute` to store the original column name.)
   * Construct the original name from `GetStructField`, `GetArrayStructFields`, 
etc.
   
   WDYT, @cloud-fan ?





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-25 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r548891083



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##
@@ -272,8 +272,14 @@ class DataSourceV2Strategy(session: SparkSession) extends 
Strategy with Predicat
   }
   DescribeTableExec(desc.output, r.table, isExtended) :: Nil
 
-case DescribeColumn(_: ResolvedTable, _, _) =>
-  throw new AnalysisException("Describing columns is not supported for v2 
tables.")
+case desc @ DescribeColumn(_: ResolvedTable, column, isExtended) =>
+  column match {

Review comment:
   Thanks for the catch. In this case, I will match against non-`Attribute` 
for nested types similar to v1:
   
https://github.com/apache/spark/blob/10b6466e91d2e954386c74bf6ab7d94f23dd6810/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala#L768-L772





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-25 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r548890864



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -1675,6 +1675,13 @@ class Analyzer(override val catalogManager: 
CatalogManager)
   // Skip the having clause here, this will be handled in 
ResolveAggregateFunctions.
   case h: UnresolvedHaving => h
 
+  case d @ DescribeColumn(rt: ResolvedTable, _, _) =>
+rt.table match {
+  // References for v1 tables are resolved in DescribeColumnCommand.
+  case _: V1Table => d

Review comment:
   Since the v1 command supports views as well, updating v1 command to take 
resolved `Attribute` doesn't solve the issue completely. We have to pass 
`Attribute` and the original column name to v1 command (or have two different 
v1 commands that take either resolved attribute or column name). Do you still 
prefer resolving attributes here for v1 tables?





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-24 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r548815634



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -344,10 +344,11 @@ case class DescribeRelation(
  */
 case class DescribeColumn(
 relation: LogicalPlan,
-colNameParts: Seq[String],
+column: NamedExpression,
 isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
   override def output: Seq[Attribute] = 
DescribeCommandSchema.describeColumnAttributes()
+  override lazy val references: AttributeSet = AttributeSet.empty

Review comment:
   Updated.





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-24 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r548791778



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveAttribute.scala
##
@@ -0,0 +1,35 @@
+/*
+ * 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.{DescribeColumn, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.V1Table
+
+/**
+ * Resolve [[UnresolvedAttribute]] in column related commands.
+ */
+case class ResolveAttribute(resolver: Resolver) extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+case r @ DescribeColumn(ResolvedTable(_, _, table), 
UnresolvedAttribute(colNameParts), _)
+if !table.isInstanceOf[V1Table] =>

Review comment:
   Not relevant anymore.





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-23 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r548302614



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -344,10 +344,11 @@ case class DescribeRelation(
  */
 case class DescribeColumn(
 relation: LogicalPlan,
-colNameParts: Seq[String],
+column: NamedExpression,
 isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
   override def output: Seq[Attribute] = 
DescribeCommandSchema.describeColumnAttributes()
+  override lazy val references: AttributeSet = AttributeSet.empty

Review comment:
   If we want to use the existing `ResolveReferences`, we can override 
`resolveChildren` here, but it still requires maintaining the same attribute 
references. So I think introducing `output` at the `ResolvedTable` is the 
cleanest approach?





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-23 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r548272691



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -344,10 +344,11 @@ case class DescribeRelation(
  */
 case class DescribeColumn(
 relation: LogicalPlan,
-colNameParts: Seq[String],
+column: NamedExpression,
 isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
   override def output: Seq[Attribute] = 
DescribeCommandSchema.describeColumnAttributes()
+  override lazy val references: AttributeSet = AttributeSet.empty

Review comment:
   This is one possible implementation of `inputSet` here:
   ```scala
 override lazy val inputSet: AttributeSet = {
   val output = relation match {
 case r: ResolvedTable => r.table.schema.toAttributes // creates new 
AttributeReference
 case _ => Seq() // should assert
   }
   AttributeSet(output)
 }
   ```
   However, when `DescribeColumn` case class is copied in `ResolveAttribute` 
(`r.copy(column = resolvedCol)`), new `AttributeReference`s are created since 
`toAttributes` is called.
   
   Any idea on the workaround (without updating `ResolvedTable` to have 
`output`)?
   
   





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-22 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r547604531



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -344,10 +344,11 @@ case class DescribeRelation(
  */
 case class DescribeColumn(
 relation: LogicalPlan,
-colNameParts: Seq[String],
+column: NamedExpression,
 isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
   override def output: Seq[Attribute] = 
DescribeCommandSchema.describeColumnAttributes()
+  override lazy val references: AttributeSet = AttributeSet.empty

Review comment:
   One disadvantage of this approach is that since `ResolvedView` only 
stores the identifier, this doesn't work for views.





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-22 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r547599204



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -344,10 +344,11 @@ case class DescribeRelation(
  */
 case class DescribeColumn(
 relation: LogicalPlan,
-colNameParts: Seq[String],
+column: NamedExpression,
 isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
   override def output: Seq[Attribute] = 
DescribeCommandSchema.describeColumnAttributes()
+  override lazy val references: AttributeSet = AttributeSet.empty

Review comment:
   One way to fix is to update `UnresolvedTable` as follow:
   ```scala
   case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, 
table: Table)
 extends LeafNode {
 override lazy val output: Seq[Attribute] = table.schema().toAttributes
   }
   ```
   Would you prefer this approach @cloud-fan ?





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-22 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r547595762



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -344,10 +344,11 @@ case class DescribeRelation(
  */
 case class DescribeColumn(
 relation: LogicalPlan,
-colNameParts: Seq[String],
+column: NamedExpression,
 isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
   override def output: Seq[Attribute] = 
DescribeCommandSchema.describeColumnAttributes()
+  override lazy val references: AttributeSet = AttributeSet.empty

Review comment:
   The issue is that in the following check, 
https://github.com/apache/spark/blob/303b8c87737fdff83c96855084c16d6504b0b50f/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala#L612-L618
   
   `missingInput` (which is `references -- inputSet`) will be non-empty since 
`references` will be from `expressions` and `inputSet` will be empty since we 
are setting `ResolvedTable`'s output to `Nil`.





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-21 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r547097927



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveAttribute.scala
##
@@ -0,0 +1,35 @@
+/*
+ * 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.{DescribeColumn, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.V1Table
+
+/**
+ * Resolve [[UnresolvedAttribute]] in column related commands.
+ */
+case class ResolveAttribute(resolver: Resolver) extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+case r @ DescribeColumn(ResolvedTable(_, _, table), 
UnresolvedAttribute(colNameParts), _)
+if !table.isInstanceOf[V1Table] =>

Review comment:
   This is so that `ResolveSessionCatalog` can pass column name parts 
directly to `DescribeColumnCommand` without resolving columns in the analyzer. 
If we want to resolve columns for both v1 and v2 here, we can introduce 
`UnresolvedAttr` and `ResolvedAttr` in `v2ResolutionPlans.scala`.





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] imback82 commented on a change in pull request #30881: [SPARK-33875][SQL] Implement DESCRIBE COLUMN for v2 tables

2020-12-21 Thread GitBox


imback82 commented on a change in pull request #30881:
URL: https://github.com/apache/spark/pull/30881#discussion_r547046976



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -344,10 +344,11 @@ case class DescribeRelation(
  */
 case class DescribeColumn(
 relation: LogicalPlan,
-colNameParts: Seq[String],
+column: NamedExpression,
 isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
   override def output: Seq[Attribute] = 
DescribeCommandSchema.describeColumnAttributes()
+  override lazy val references: AttributeSet = AttributeSet.empty

Review comment:
   This is required since `column` is an expression, 
`QueryPlan.expressions` will pick up this field.

##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
##
@@ -149,6 +149,9 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
   case AlterTable(_, _, u: UnresolvedV2Relation, _) =>
 failAnalysis(s"Table not found: ${u.originalNameParts.quoted}")
 
+  case DescribeColumn(_: ResolvedTable, UnresolvedAttribute(colNameParts), 
_) =>
+failAnalysis(s"Column not found: ${colNameParts.quoted}")

Review comment:
   This check should come before expression checks. Otherwise, we will get 
`org.apache.spark.sql.AnalysisException: cannot resolve '`invalid_col`' given 
input columns: []` and this is misleading since input columns are printed as 
empty.

##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -344,10 +344,11 @@ case class DescribeRelation(
  */
 case class DescribeColumn(
 relation: LogicalPlan,
-colNameParts: Seq[String],
+column: NamedExpression,
 isExtended: Boolean) extends Command {
   override def children: Seq[LogicalPlan] = Seq(relation)
   override def output: Seq[Attribute] = 
DescribeCommandSchema.describeColumnAttributes()
+  override lazy val references: AttributeSet = AttributeSet.empty

Review comment:
   Another way to approach is to use `LogicalPlan` instead of 
`NamedExpression`, but I went with this approach to reuse `UnresolvedAttribute`.
   
   cc @cloud-fan 





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