[GitHub] [spark] cloud-fan commented on a change in pull request #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-12-09 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##
@@ -288,19 +289,23 @@ final class DataStreamWriter[T] private[sql](ds: 
Dataset[T]) {
* Note, currently the new table creation by this API doesn't fully 
cover the V2 table.
* TODO (SPARK-33638): Full support of v2 table creation
*/
-  val cmd = CreateTableStatement(
-originalMultipartIdentifier,
-df.schema.asNullable,
-partitioningColumns.getOrElse(Nil).asTransforms.toSeq,
+  val tableProperties = TableSpec(

Review comment:
   typo: this should be `tableSpec`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-29 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
##
@@ -26,12 +26,12 @@ import org.mockito.invocation.InvocationOnMock
 
 import org.apache.spark.sql.{AnalysisException, SaveMode}
 import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
-import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AnalysisTest, 
Analyzer, EmptyFunctionRegistry, NoSuchTableException, ResolvedFieldName, 
ResolvedTable, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, 
UnresolvedSubqueryColumnAliases, UnresolvedTable}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AnalysisTest, 
Analyzer, EmptyFunctionRegistry, NoSuchTableException, ResolvedDBObjectName, 
ResolvedFieldName, ResolvedTable, ResolveSessionCatalog, UnresolvedAttribute, 
UnresolvedRelation, UnresolvedSubqueryColumnAliases, UnresolvedTable}
 import org.apache.spark.sql.catalyst.catalog.{BucketSpec, 
CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, 
SessionCatalog}
 import org.apache.spark.sql.catalyst.expressions.{AnsiCast, 
AttributeReference, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, 
Literal, StringLiteral}
 import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke
 import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException}
-import org.apache.spark.sql.catalyst.plans.logical.{AlterColumn, 
AnalysisOnlyCommand, AppendData, Assignment, CreateTableAsSelect, 
CreateTableStatement, CreateV2Table, DeleteAction, DeleteFromTable, 
DescribeRelation, DropTable, InsertAction, LocalRelation, LogicalPlan, 
MergeIntoTable, OneRowRelation, Project, SetTableLocation, SetTableProperties, 
ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, 
UpdateTable}
+import org.apache.spark.sql.catalyst.plans.logical.{AlterColumn, 
AnalysisOnlyCommand, AppendData, Assignment, CreateTable => 
CatalystCreateTable, CreateTableAsSelect, DeleteAction, DeleteFromTable, 
DescribeRelation, DropTable, InsertAction, LocalRelation, LogicalPlan, 
MergeIntoTable, OneRowRelation, Project, SetTableLocation, SetTableProperties, 
ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, 
UpdateTable}

Review comment:
   nit: can we change `import 
org.apache.spark.sql.execution.datasources.CreateTable` to `import 
org.apache.spark.sql.execution.datasources.CreateTableV1` and keep the 
`CreateTable` here unchanged?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-29 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
##
@@ -819,6 +820,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product with Tre
   redactMapString(map.asCaseSensitiveMap().asScala, maxFields)
 case map: Map[_, _] =>
   redactMapString(map, maxFields)
+case t: TableSpec =>
+  redactMapString(t.options, maxFields)
+  redactMapString(t.properties, maxFields)

Review comment:
   I think we need to return a copy of `TableSpec` with options and 
properties redacted.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-25 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
##
@@ -481,19 +481,26 @@ class PlanResolutionSuite extends AnalysisTest {
   "other" -> "20")
 
 parseAndResolve(sql) match {
-  case create: CreateV2Table =>
-assert(create.catalog.name == "testcat")
-assert(create.tableName == Identifier.of(Array("mydb"), "table_name"))
+  case create: CatalystCreateTable =>
+assert(create.name.asInstanceOf[ResolvedDBObjectName].catalog.name == 
"testcat")
+
assert(create.name.asInstanceOf[ResolvedDBObjectName].nameParts.mkString(".") ==
+  "mydb.table_name")
 assert(create.tableSchema == new StructType()
 .add("id", LongType)
 .add("description", StringType)
 .add("point", new StructType().add("x", DoubleType).add("y", 
DoubleType)))
 assert(create.partitioning.isEmpty)
-assert(create.properties == expectedProperties)

Review comment:
   do we have end-to-end test for CREATE TABLE? we can test it there.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-25 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
##
@@ -481,19 +481,26 @@ class PlanResolutionSuite extends AnalysisTest {
   "other" -> "20")
 
 parseAndResolve(sql) match {
-  case create: CreateV2Table =>
-assert(create.catalog.name == "testcat")
-assert(create.tableName == Identifier.of(Array("mydb"), "table_name"))
+  case create: CatalystCreateTable =>
+assert(create.name.asInstanceOf[ResolvedDBObjectName].catalog.name == 
"testcat")
+
assert(create.name.asInstanceOf[ResolvedDBObjectName].nameParts.mkString(".") ==
+  "mydb.table_name")
 assert(create.tableSchema == new StructType()
 .add("id", LongType)
 .add("description", StringType)
 .add("point", new StructType().add("x", DoubleType).add("y", 
DoubleType)))
 assert(create.partitioning.isEmpty)
-assert(create.properties == expectedProperties)

Review comment:
   I think we don't need to test it now as it's not part of the resolution 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-25 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala
##
@@ -32,10 +33,16 @@ case class CreateTableExec(
 identifier: Identifier,
 tableSchema: StructType,
 partitioning: Seq[Transform],
-tableProperties: Map[String, String],
+tableSpec: TableSpec,
 ignoreIfExists: Boolean) extends LeafV2CommandExec {
   import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
+  val props = CatalogV2Util.convertTableProperties(

Review comment:
   let's not expose this as it's intermediate
   ```
   val tableProperties = {
 val props = ...
 CatalogV2Util.withDefaultOwnership(props)
   }
   ```

##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala
##
@@ -32,10 +33,16 @@ case class CreateTableExec(
 identifier: Identifier,
 tableSchema: StructType,
 partitioning: Seq[Transform],
-tableProperties: Map[String, String],
+tableSpec: TableSpec,
 ignoreIfExists: Boolean) extends LeafV2CommandExec {
   import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 
+  val props = CatalogV2Util.convertTableProperties(

Review comment:
   let's not expose this as it's intermediate. how about
   ```
   val tableProperties = {
 val props = ...
 CatalogV2Util.withDefaultOwnership(props)
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-25 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##
@@ -3402,7 +3402,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   }
 
   /**
-   * Create a table, returning a [[CreateTableStatement]] logical plan.
+   * Create a table, returning a [[CreateTable]] or [[CreateV2Table]] logical 
plan.

Review comment:
   ```suggestion
  * Create a table, returning a [[CreateTable]] or 
[[CreateTableAsSelectStatement]] logical plan.
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-25 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##
@@ -3469,9 +3469,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   case _ =>
 // Note: table schema includes both the table columns list and the 
partition columns
 // with data type.
+val tableSpec = TableSpec(bucketSpec, properties, provider, options, 
location, comment,
+  serdeInfo, external)
 val schema = StructType(columns ++ partCols)
-CreateTableStatement(table, schema, partitioning, bucketSpec, 
properties, provider,
-  options, location, comment, serdeInfo, external = external, 
ifNotExists = ifNotExists)
+CreateTable(
+  UnresolvedDBObjectName(
+table,
+isNamespace = false),

Review comment:
   ```suggestion
 UnresolvedDBObjectName(table, isNamespace = false),
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-24 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -189,17 +190,35 @@ trait V2CreateTablePlan extends LogicalPlan {
   def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan
 }
 
+trait V2CreateTablePlanX extends LogicalPlan {

Review comment:
   Do we have to add this? Can we let the new `CreateTable` extends 
`V2CreateTablePlan` and implement `tableName`?
   ```
   def tableName = {
 assert(name.isResolved)
 name.asInstanceOf[ResolvedDBObjectName].nameParts.asIdentifier
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-24 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##
@@ -156,9 +156,16 @@ class DataSourceV2Strategy(session: SparkSession) extends 
Strategy with Predicat
   }
   WriteToDataSourceV2Exec(writer, invalidateCacheFunc, planLater(query), 
customMetrics) :: Nil
 
-case CreateV2Table(catalog, ident, schema, parts, props, ifNotExists) =>
+case CreateTable(ResolvedDBObjectName(catalog, ident), schema, 
partitioning, bucketSpec,
+tableSpec, ifNotExists) =>
+  val props = CatalogV2Util.convertTableProperties(

Review comment:
   I think it's better to keep the strategy rule simple and move the logic 
of generating table properties into `CreateTableExec`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-23 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
##
@@ -263,6 +265,51 @@ case class PreprocessTableCreation(sparkSession: 
SparkSession) extends Rule[Logi
 
 create.withPartitioning(normalizedPartitions)
   }
+
+case create: V2CreateTablePlanX if create.childrenResolved =>
+  val schema = create.tableSchema
+  val partitioning = create.partitioning
+  val name = create.name.asInstanceOf[ResolvedDBObjectName].nameParts
+  val identifier = if (name.length == 2) {
+Identifier.of(Array(name(0)), name(1))
+  } else {
+Identifier.of(Array.empty, name(0))

Review comment:
   what if the `name` has more than 2 parts?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-23 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
##
@@ -25,9 +25,11 @@ import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.expressions.{Expression, 
InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering}
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.Identifier
 import org.apache.spark.sql.connector.expressions.{FieldReference, 
RewritableTransform}
 import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.execution.command.DDLUtils
+import org.apache.spark.sql.execution.datasources.{CreateTable => 
DataSourceCreateTable}

Review comment:
   ```suggestion
   import org.apache.spark.sql.execution.datasources.{CreateTable => 
CreateTableV1}
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-23 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
##
@@ -819,6 +820,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product with Tre
   redactMapString(map.asCaseSensitiveMap().asScala, maxFields)
 case map: Map[_, _] =>
   redactMapString(map, maxFields)
+case t: TableSpec =>
+  redactMapString(t.options, maxFields)

Review comment:
   why do we only deal with `t.options` here?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-23 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -190,17 +191,32 @@ trait V2CreateTablePlan extends LogicalPlan {
   def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan
 }
 
+trait V2CreateTablePlanX extends LogicalPlan {
+  def name: LogicalPlan
+  def partitioning: Seq[Transform]
+  def tableSchema: StructType
+
+  /**
+   * Creates a copy of this node with the new partitioning transforms. This 
method is used to
+   * rewrite the partition transforms normalized according to the table schema.
+   */
+  def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlanX
+}
+
 /**
  * Create a new table with a v2 catalog.
  */
-case class CreateV2Table(
-catalog: TableCatalog,
-tableName: Identifier,
+case class CreateTable(
+name: LogicalPlan,
 tableSchema: StructType,
 partitioning: Seq[Transform],
-properties: Map[String, String],
-ignoreIfExists: Boolean) extends LeafCommand with V2CreateTablePlan {
-  override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan 
= {
+bucketSpec: Option[BucketSpec],

Review comment:
   nit: shall we move `bucketSpec` to `TableSpec`? According to the v2 
createTable API, only schema and partitioning are top-level.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-23 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
##
@@ -143,25 +143,25 @@ class ResolveSessionCatalog(val catalogManager: 
CatalogManager)
 
 // For CREATE TABLE [AS SELECT], we should use the v1 command if the 
catalog is resolved to the
 // session catalog and the table provider is not v2.
-case c @ CreateTableStatement(
- SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _, 
_) =>
+case c @ CreateV2Table(ResolvedDBObjectName(catalog, name), _, _, _, _, _) 
=>

Review comment:
   I think we should only change the plan if the catalog is session 
catalog. The `CreateTableExec` should be responsible to convert `TableSpec` to 
table properties.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-23 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##
@@ -1090,3 +1106,12 @@ case class DropIndex(
   override protected def withNewChildInternal(newChild: LogicalPlan): 
DropIndex =
 copy(table = newChild)
 }
+
+case class TableProperties(

Review comment:
   This name is a bit misleading. How about `TableSpec`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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 #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework

2021-11-23 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##
@@ -3469,9 +3469,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   case _ =>
 // Note: table schema includes both the table columns list and the 
partition columns
 // with data type.
+val tableProperties = TableProperties(properties, provider, options, 
location, comment,
+  serdeInfo, external)
 val schema = StructType(columns ++ partCols)
-CreateTableStatement(table, schema, partitioning, bucketSpec, 
properties, provider,
-  options, location, comment, serdeInfo, external = external, 
ifNotExists = ifNotExists)
+CreateV2Table(

Review comment:
   The naming is a bit hard here. The best name should be `CreateTable`, 
but it's already taken. `CreateV2Table` is not proper here as this can also 
create v1 tables.
   
   Given that we can will migrate all the commands eventually, the existing 
`CreateTable` will be removed at the end. How about we still use the name 
`CreateTable` here? i.e. add a new `CreateTable` logical plan in 
`org.apache.spark.sql.catalyst.plans.logical`, which does not conflict with the 
existing `CreateTable`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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