[GitHub] [spark] cloud-fan commented on a change in pull request #34060: [SPARK-36850][SQL] Migrate CreateTableStatement to v2 command framework
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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