cloud-fan commented on code in PR #49772:
URL: https://github.com/apache/spark/pull/49772#discussion_r1954022933


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDDLCommandStringTypes.scala:
##########
@@ -18,80 +18,48 @@
 package org.apache.spark.sql.catalyst.analysis
 
 import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, Literal}
-import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumns, 
AlterColumnSpec, AlterViewAs, ColumnDefinition, CreateView, LogicalPlan, 
QualifiedColType, ReplaceColumns, V1CreateTablePlan, V2CreateTablePlan}
-import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumns, 
AlterColumnSpec, AlterTableCommand, AlterViewAs, ColumnDefinition, CreateTable, 
CreateView, LogicalPlan, QualifiedColType, ReplaceColumns, V1CreateTablePlan, 
V2CreateTablePlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.TableCatalog
 import org.apache.spark.sql.types.{DataType, StringType}
 
 /**
- * Resolves default string types in queries and commands. For queries, the 
default string type is
- * determined by the session's default string type. For DDL, the default 
string type is the
- * default type of the object (table -> schema -> catalog). However, this is 
not implemented yet.
- * So, we will just use UTF8_BINARY for now.
+ * Resolves string types in DDL commands, where the string type inherits the
+ * collation from the corresponding object (table/view -> schema -> catalog).
  */
-object ResolveDefaultStringTypes extends Rule[LogicalPlan] {
+object ResolveDDLCommandStringTypes extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = {
-    val newPlan = apply0(plan)
-    if (plan.ne(newPlan)) {
-      // Due to how tree transformations work and StringType object being 
equal to
-      // StringType("UTF8_BINARY"), we need to transform the plan twice
-      // to ensure the correct results for occurrences of default string type.
-      val finalPlan = apply0(newPlan)
-      RuleExecutor.forceAdditionalIteration(finalPlan)
-      finalPlan
-    } else {
-      newPlan
-    }
-  }
-
-  private def apply0(plan: LogicalPlan): LogicalPlan = {
     if (isDDLCommand(plan)) {
       transformDDL(plan)
     } else {
-      transformPlan(plan, sessionDefaultStringType)
+      // For non-DDL commands no need to do any further resolution of string 
types
+      plan
     }
   }
 
-  /**
-   * Returns whether any of the given `plan` needs to have its
-   * default string type resolved.
-   */
-  def needsResolution(plan: LogicalPlan): Boolean = {
-    if (!isDDLCommand(plan) && isDefaultSessionCollationUsed) {
-      return false
+  /** Default collation used, if object level collation is not provided */
+  private def defaultCollation: String = "UTF8_BINARY"
+
+  /** Returns the string type that should be used in a given DDL command */
+  private def stringTypeForDDLCommand(table: LogicalPlan): StringType = {
+    table match {
+      case createTable: CreateTable if 
createTable.tableSpec.collation.isDefined =>
+        StringType(createTable.tableSpec.collation.get)
+      case createView: CreateView if createView.collation.isDefined =>
+        StringType(createView.collation.get)
+      case alterTable: AlterTableCommand if alterTable.table.resolved =>
+        val collation = Option(alterTable
+          .table.asInstanceOf[ResolvedTable]
+          .table.properties.get(TableCatalog.PROP_COLLATION))
+        if (collation.isDefined) {
+          StringType(collation.get)
+        } else {
+          StringType(defaultCollation)
+        }
+      case _ => StringType(defaultCollation)
     }
-
-    plan.exists(node => needsResolution(node.expressions))
-  }
-
-  /**
-   * Returns whether any of the given `expressions` needs to have its
-   * default string type resolved.
-   */
-  def needsResolution(expressions: Seq[Expression]): Boolean = {
-    expressions.exists(needsResolution)
   }
 
-  /**
-   * Returns whether the given `expression` needs to have its
-   * default string type resolved.
-   */
-  def needsResolution(expression: Expression): Boolean = {
-    expression.exists(e => transformExpression.isDefinedAt(e))
-  }
-
-  private def isDefaultSessionCollationUsed: Boolean = conf.defaultStringType 
== StringType
-
-  /**
-   * Returns the default string type that should be used in a given DDL 
command (for now always
-   * UTF8_BINARY).
-   */
-  private def stringTypeForDDLCommand(table: LogicalPlan): StringType =
-    StringType("UTF8_BINARY")
-
-  /** Returns the session default string type */
-  private def sessionDefaultStringType: StringType =
-    StringType(conf.defaultStringType.collationId)
-
   private def isDDLCommand(plan: LogicalPlan): Boolean = plan exists {
     case _: AddColumns | _: ReplaceColumns | _: AlterColumns => true
     case _ => isCreateOrAlterPlan(plan)

Review Comment:
   For CTAS, shall we update the table default collation for the input query or 
not?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDDLCommandStringTypes.scala:
##########
@@ -18,80 +18,48 @@
 package org.apache.spark.sql.catalyst.analysis
 
 import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, Literal}
-import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumns, 
AlterColumnSpec, AlterViewAs, ColumnDefinition, CreateView, LogicalPlan, 
QualifiedColType, ReplaceColumns, V1CreateTablePlan, V2CreateTablePlan}
-import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumns, 
AlterColumnSpec, AlterTableCommand, AlterViewAs, ColumnDefinition, CreateTable, 
CreateView, LogicalPlan, QualifiedColType, ReplaceColumns, V1CreateTablePlan, 
V2CreateTablePlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.TableCatalog
 import org.apache.spark.sql.types.{DataType, StringType}
 
 /**
- * Resolves default string types in queries and commands. For queries, the 
default string type is
- * determined by the session's default string type. For DDL, the default 
string type is the
- * default type of the object (table -> schema -> catalog). However, this is 
not implemented yet.
- * So, we will just use UTF8_BINARY for now.
+ * Resolves string types in DDL commands, where the string type inherits the
+ * collation from the corresponding object (table/view -> schema -> catalog).
  */
-object ResolveDefaultStringTypes extends Rule[LogicalPlan] {
+object ResolveDDLCommandStringTypes extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = {
-    val newPlan = apply0(plan)
-    if (plan.ne(newPlan)) {
-      // Due to how tree transformations work and StringType object being 
equal to
-      // StringType("UTF8_BINARY"), we need to transform the plan twice
-      // to ensure the correct results for occurrences of default string type.
-      val finalPlan = apply0(newPlan)
-      RuleExecutor.forceAdditionalIteration(finalPlan)
-      finalPlan
-    } else {
-      newPlan
-    }
-  }
-
-  private def apply0(plan: LogicalPlan): LogicalPlan = {
     if (isDDLCommand(plan)) {
       transformDDL(plan)
     } else {
-      transformPlan(plan, sessionDefaultStringType)
+      // For non-DDL commands no need to do any further resolution of string 
types
+      plan
     }
   }
 
-  /**
-   * Returns whether any of the given `plan` needs to have its
-   * default string type resolved.
-   */
-  def needsResolution(plan: LogicalPlan): Boolean = {
-    if (!isDDLCommand(plan) && isDefaultSessionCollationUsed) {
-      return false
+  /** Default collation used, if object level collation is not provided */
+  private def defaultCollation: String = "UTF8_BINARY"
+
+  /** Returns the string type that should be used in a given DDL command */
+  private def stringTypeForDDLCommand(table: LogicalPlan): StringType = {
+    table match {
+      case createTable: CreateTable if 
createTable.tableSpec.collation.isDefined =>
+        StringType(createTable.tableSpec.collation.get)
+      case createView: CreateView if createView.collation.isDefined =>
+        StringType(createView.collation.get)
+      case alterTable: AlterTableCommand if alterTable.table.resolved =>
+        val collation = Option(alterTable
+          .table.asInstanceOf[ResolvedTable]
+          .table.properties.get(TableCatalog.PROP_COLLATION))
+        if (collation.isDefined) {
+          StringType(collation.get)
+        } else {
+          StringType(defaultCollation)
+        }
+      case _ => StringType(defaultCollation)
     }
-
-    plan.exists(node => needsResolution(node.expressions))
-  }
-
-  /**
-   * Returns whether any of the given `expressions` needs to have its
-   * default string type resolved.
-   */
-  def needsResolution(expressions: Seq[Expression]): Boolean = {
-    expressions.exists(needsResolution)
   }
 
-  /**
-   * Returns whether the given `expression` needs to have its
-   * default string type resolved.
-   */
-  def needsResolution(expression: Expression): Boolean = {
-    expression.exists(e => transformExpression.isDefinedAt(e))
-  }
-
-  private def isDefaultSessionCollationUsed: Boolean = conf.defaultStringType 
== StringType
-
-  /**
-   * Returns the default string type that should be used in a given DDL 
command (for now always
-   * UTF8_BINARY).
-   */
-  private def stringTypeForDDLCommand(table: LogicalPlan): StringType =
-    StringType("UTF8_BINARY")
-
-  /** Returns the session default string type */
-  private def sessionDefaultStringType: StringType =
-    StringType(conf.defaultStringType.collationId)
-
   private def isDDLCommand(plan: LogicalPlan): Boolean = plan exists {
     case _: AddColumns | _: ReplaceColumns | _: AlterColumns => true
     case _ => isCreateOrAlterPlan(plan)

Review Comment:
   For CTAS, shall we respect the table default collation for the input query 
or not?



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

Reply via email to