This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 6534a3398ae9 [SPARK-47102][SQL] Add the `COLLATION_ENABLED` config flag
6534a3398ae9 is described below

commit 6534a3398ae9f4e14b02f4821bccde4b3671dc17
Author: Mihailo Milosevic <mihailo.milose...@databricks.com>
AuthorDate: Tue Mar 5 15:11:26 2024 +0300

    [SPARK-47102][SQL] Add the `COLLATION_ENABLED` config flag
    
    ### What changes were proposed in this pull request?
    This PR adds `COLLATION_ENABLED` config to `SQLConf` and introduces new 
error class `UNSUPPORTED_FEATURE.COLLATION` to appropriately report error on 
usage of feature under development.
    
    Closes #45218
    
    ### Why are the changes needed?
    We want to make collations configurable on this flag. These changes disable 
usage of `collate` and `collation` functions, along with any `COLLATE` syntax 
when the flag is set to false. By default, the flag is set to false.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes. It introduces new error along with an appropriate message.
    
    ### How was this patch tested?
    ```
    ./build/mvn -Dtest=none 
-DwildcardSuites=org.apache.spark.sql.errors.QueryCompilationErrorsSuite test
    ./build/mvn -Dtest=none 
-DwildcardSuites=org.apache.spark.sql.catalyst.expressions.CollationExpressionSuite
 test
    ./build/mvn -Dtest=none 
-DwildcardSuites=org.apache.spark.sql.CollationSuite test
    ```
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #45285 from mihailom-db/SPARK-47102.
    
    Authored-by: Mihailo Milosevic <mihailo.milose...@databricks.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../src/main/resources/error/error-classes.json    |  5 ++++
 ...r-conditions-unsupported-feature-error-class.md |  4 +++
 .../expressions/collationExpressions.scala         | 27 +++++++++++++++++--
 .../spark/sql/catalyst/parser/AstBuilder.scala     |  7 +++++
 .../spark/sql/errors/QueryCompilationErrors.scala  |  6 +++++
 .../org/apache/spark/sql/internal/SQLConf.scala    | 10 +++++++
 .../spark/sql/execution/datasources/rules.scala    | 21 ++++++++++++++-
 .../sql/internal/BaseSessionStateBuilder.scala     |  1 +
 .../sql/errors/QueryCompilationErrorsSuite.scala   | 31 ++++++++++++++++++++++
 .../spark/sql/hive/HiveSessionStateBuilder.scala   |  1 +
 10 files changed, 110 insertions(+), 3 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 7cf3e9c533ca..c61f348e63d8 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3921,6 +3921,11 @@
           "Catalog <catalogName> does not support <operation>."
         ]
       },
+      "COLLATION" : {
+        "message" : [
+          "Collation is not yet supported."
+        ]
+      },
       "COMBINATION_QUERY_RESULT_CLAUSES" : {
         "message" : [
           "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY."
diff --git a/docs/sql-error-conditions-unsupported-feature-error-class.md 
b/docs/sql-error-conditions-unsupported-feature-error-class.md
index 8d42ecdce790..e580ecc63b18 100644
--- a/docs/sql-error-conditions-unsupported-feature-error-class.md
+++ b/docs/sql-error-conditions-unsupported-feature-error-class.md
@@ -58,6 +58,10 @@ The ANALYZE TABLE command does not support views.
 
 Catalog `<catalogName>` does not support `<operation>`.
 
+## COLLATION
+
+Collation is not yet supported.
+
 ## COMBINATION_QUERY_RESULT_CLAUSES
 
 Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY.
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala
index a2faca95dfbc..e51d9a67b166 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala
@@ -22,8 +22,10 @@ import 
org.apache.spark.sql.catalyst.analysis.ExpressionBuilder
 import org.apache.spark.sql.catalyst.expressions.codegen._
 import org.apache.spark.sql.catalyst.util.CollationFactory
 import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
+// scalastyle:off line.contains.tab
 @ExpressionDescription(
   usage = "_FUNC_(expr, collationName) - Marks a given expression with the 
specified collation.",
   arguments = """
@@ -33,13 +35,24 @@ import org.apache.spark.sql.types._
   """,
   examples = """
     Examples:
+      > SET spark.sql.collation.enabled=true;
+      spark.sql.collation.enabled      true
       > SELECT COLLATION('Spark SQL' _FUNC_ 'UCS_BASIC_LCASE');
-       UCS_BASIC_LCASE
+      UCS_BASIC_LCASE
+      > SET spark.sql.collation.enabled=false;
+      spark.sql.collation.enabled      false
   """,
   since = "4.0.0",
   group = "string_funcs")
+// scalastyle:on line.contains.tab
 object CollateExpressionBuilder extends ExpressionBuilder {
   override def build(funcName: String, expressions: Seq[Expression]): 
Expression = {
+    // We need to throw collationNotEnabledError before unexpectedNullError
+    // and nonFoldableArgumentError, as we do not want user to see misleading
+    // messages that collation is enabled
+    if (!SQLConf.get.collationEnabled) {
+      throw QueryCompilationErrors.collationNotEnabledError()
+    }
     expressions match {
       case Seq(e: Expression, collationExpr: Expression) =>
         (collationExpr.dataType, collationExpr.foldable) match {
@@ -80,15 +93,25 @@ case class Collate(child: Expression, collationName: String)
     defineCodeGen(ctx, ev, (in) => in)
 }
 
+// scalastyle:off line.contains.tab
 @ExpressionDescription(
   usage = "_FUNC_(expr) - Returns the collation name of a given expression.",
+  arguments = """
+    Arguments:
+      * expr - String expression to perform collation on.
+  """,
   examples = """
     Examples:
+      > SET spark.sql.collation.enabled=true;
+      spark.sql.collation.enabled      true
       > SELECT _FUNC_('Spark SQL');
-       UCS_BASIC
+      UCS_BASIC
+      > SET spark.sql.collation.enabled=false;
+      spark.sql.collation.enabled      false
   """,
   since = "4.0.0",
   group = "string_funcs")
+// scalastyle:on line.contains.tab
 case class Collation(child: Expression) extends UnaryExpression with 
RuntimeReplaceable {
   override def dataType: DataType = StringType
   override protected def withNewChildInternal(newChild: Expression): Collation 
= copy(newChild)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 096dbe840ead..48d8b8ad8f4d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -2190,6 +2190,13 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
     Collate(expression(ctx.primaryExpression), collationName)
   }
 
+  override def visitCollateClause(ctx: CollateClauseContext): String = 
withOrigin(ctx) {
+    if (!SQLConf.get.collationEnabled) {
+      throw QueryCompilationErrors.collationNotEnabledError()
+    }
+    string(visitStringLit(ctx.stringLit))
+  }
+
   /**
    * Create a [[Cast]] expression.
    */
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 38f2228f3389..95a66f015729 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -297,6 +297,12 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase with Compilat
     )
   }
 
+  def collationNotEnabledError(): Throwable = {
+    new AnalysisException(
+      errorClass = "UNSUPPORTED_FEATURE.COLLATION",
+      messageParameters = Map.empty)
+  }
+
   def unresolvedUsingColForJoinError(
       colName: String, suggestion: String, side: String): Throwable = {
     new AnalysisException(
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 04b392d0c44f..b262cb29c02e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -753,6 +753,14 @@ object SQLConf {
       .checkValue(_ > 0, "The initial number of partitions must be positive.")
       .createOptional
 
+  lazy val COLLATION_ENABLED =
+    buildConf("spark.sql.collation.enabled")
+      .doc("Collations feature is under development and its use should be done 
under this" +
+        "feature flag.")
+      .version("4.0.0")
+      .booleanConf
+      .createWithDefault(Utils.isTesting)
+
   val FETCH_SHUFFLE_BLOCKS_IN_BATCH =
     buildConf("spark.sql.adaptive.fetchShuffleBlocksInBatch")
       .internal()
@@ -4962,6 +4970,8 @@ class SQLConf extends Serializable with Logging with 
SqlApiConf {
     }
   }
 
+  def collationEnabled: Boolean = getConf(COLLATION_ENABLED)
+
   def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED)
 
   def adaptiveExecutionLogLevel: String = getConf(ADAPTIVE_EXECUTION_LOG_LEVEL)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
index fceef89c3111..c9ad6dcaac8f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
@@ -22,7 +22,7 @@ import java.util.Locale
 import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession}
 import org.apache.spark.sql.catalyst.analysis._
 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.expressions.{Collate, Collation, 
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.catalyst.types.DataTypeUtils.toAttributes
@@ -32,6 +32,7 @@ import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.execution.command.DDLUtils
 import org.apache.spark.sql.execution.datasources.{CreateTable => 
CreateTableV1}
 import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.sources.InsertableRelation
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.util.PartitioningUtils.normalizePartitionSpec
@@ -594,3 +595,21 @@ case class QualifyLocationWithWarehouse(catalog: 
SessionCatalog) extends Rule[Lo
       c.copy(tableDesc = newTable)
   }
 }
+
+object CollationCheck extends (LogicalPlan => Unit) {
+  def apply(plan: LogicalPlan): Unit = {
+    plan.foreach {
+      case operator: LogicalPlan =>
+        operator.expressions.foreach(_.foreach(
+          e =>
+            if (isCollationExpression(e) && !SQLConf.get.collationEnabled) {
+              throw QueryCompilationErrors.collationNotEnabledError()
+            }
+          )
+        )
+    }
+  }
+
+  private def isCollationExpression(expression: Expression): Boolean =
+    expression.isInstanceOf[Collation] || expression.isInstanceOf[Collate]
+}
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
index 00c72294ca07..63c0d116ba3a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
@@ -223,6 +223,7 @@ abstract class BaseSessionStateBuilder(
         HiveOnlyCheck +:
         TableCapabilityCheck +:
         CommandCheck +:
+        CollationCheck +:
         customCheckRules
   }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
index 7554f8563393..a9506ad5422d 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
@@ -964,6 +964,37 @@ class QueryCompilationErrorsSuite
         "className" -> "org.apache.spark.sql.catalyst.expressions.UnsafeRow"))
   }
 
+  test("SPARK-47102: the collation feature is off without collate builder 
call") {
+    withSQLConf(SQLConf.COLLATION_ENABLED.key -> "false") {
+      Seq(
+        "CREATE TABLE t(col STRING COLLATE 'UNICODE_CI') USING parquet",
+        "CREATE TABLE t(col STRING COLLATE 'UNKNOWN_COLLATION_STRING') USING 
parquet",
+        "SELECT 'aaa' COLLATE 'UNICODE_CI'",
+        "select collation('aaa')"
+      ).foreach { sqlText =>
+        checkError(
+          exception = intercept[AnalysisException](sql(sqlText)),
+          errorClass = "UNSUPPORTED_FEATURE.COLLATION")
+      }
+    }
+  }
+
+  test("SPARK-47102: the collation feature is off with collate builder call") {
+    withSQLConf(SQLConf.COLLATION_ENABLED.key -> "false") {
+      Seq(
+        "SELECT collate('aaa', 'UNICODE_CI')",
+        "SELECT collate('aaa', 'UNKNOWN_COLLATION_STRING')"
+      ).foreach { sqlText =>
+        checkError(
+          exception = intercept[AnalysisException](sql(sqlText)),
+          errorClass = "UNSUPPORTED_FEATURE.COLLATION",
+          parameters = Map.empty,
+          context = ExpectedContext(
+            fragment = sqlText.substring(7), start = 7, stop = sqlText.length 
- 1))
+      }
+    }
+  }
+
   test("INTERNAL_ERROR: Convert unsupported data type from Spark to Parquet") {
     val converter = new SparkToParquetSchemaConverter
     val dummyDataType = new DataType {
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
index 32100d060b09..416299b189cd 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
@@ -113,6 +113,7 @@ class HiveSessionStateBuilder(
         PreReadCheck +:
         TableCapabilityCheck +:
         CommandCheck +:
+        CollationCheck +:
         customCheckRules
   }
 


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

Reply via email to