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 7a428e8ed45a9..d45ca533f9392 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 @@ -36,24 +36,14 @@ import org.apache.spark.sql.types._ """, examples = """ Examples: - > SET spark.sql.collation.enabled=true; - spark.sql.collation.enabled true > SELECT COLLATION('Spark SQL' _FUNC_ UTF8_LCASE); UTF8_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 { @@ -107,12 +97,8 @@ case class Collate(child: Expression, collationName: String) """, examples = """ Examples: - > SET spark.sql.collation.enabled=true; - spark.sql.collation.enabled true > SELECT _FUNC_('Spark SQL'); UTF8_BINARY - > SET spark.sql.collation.enabled=false; - spark.sql.collation.enabled false """, since = "4.0.0", group = "string_funcs") 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 a505a530b9591..cf3b3fa16f1c9 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 @@ -2378,9 +2378,6 @@ class AstBuilder extends DataTypeAstBuilder } override def visitCollateClause(ctx: CollateClauseContext): String = withOrigin(ctx) { - if (!SQLConf.get.collationEnabled) { - throw QueryCompilationErrors.collationNotEnabledError() - } ctx.identifier.getText } 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 12ce66bc9ffd4..cf801e3caacb2 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 @@ -351,12 +351,6 @@ 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 4202f2453c924..0d61bc7f19eeb 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 @@ -759,14 +759,6 @@ 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 DEFAULT_COLLATION = buildConf(SqlApiConfHelper.DEFAULT_COLLATION) .doc("Sets default collation to use for string literals, parameter markers or the string" + @@ -5413,8 +5405,6 @@ class SQLConf extends Serializable with Logging with SqlApiConf { } } - def collationEnabled: Boolean = getConf(COLLATION_ENABLED) - override def defaultStringType: StringType = { if (getConf(DEFAULT_COLLATION).toUpperCase(Locale.ROOT) == "UTF8_BINARY") { StringType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index dc5b13b9b0ae9..8566bd08ec514 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -150,9 +150,6 @@ class SparkSqlAstBuilder extends AstBuilder { * }}} */ override def visitSetCollation(ctx: SetCollationContext): LogicalPlan = withOrigin(ctx) { - if (!SQLConf.get.collationEnabled) { - throw QueryCompilationErrors.collationNotEnabledError() - } val key = SQLConf.DEFAULT_COLLATION.key SetCommand(Some(key -> Some(ctx.identifier.getText.toUpperCase(Locale.ROOT)))) } 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 f9fe5390e16b0..29385904a7525 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 @@ -26,7 +26,7 @@ import org.apache.spark.SparkIllegalArgumentException 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.{Collate, Collation, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} +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.catalyst.types.DataTypeUtils.toAttributes @@ -37,7 +37,6 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.command.ViewHelper.generateViewProperties 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.{StructField, StructType} import org.apache.spark.sql.util.PartitioningUtils.normalizePartitionSpec @@ -641,25 +640,6 @@ case class QualifyLocationWithWarehouse(catalog: SessionCatalog) extends Rule[Lo } } -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] -} - - /** * This rule checks for references to views WITH SCHEMA [TYPE] EVOLUTION and synchronizes the * catalog if evolution was detected. 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 4660970814e21..99287bddb5104 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,7 +223,6 @@ abstract class BaseSessionStateBuilder( HiveOnlyCheck +: TableCapabilityCheck +: CommandCheck +: - CollationCheck +: ViewSyncSchemaToMetaStore +: 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 958d2b0130d8b..371a615828de3 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 @@ -878,37 +878,6 @@ 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/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 404ec865c1b00..05cd9800bdf21 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -517,14 +517,6 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { "confName" -> "spark.sql.session.collation.default", "proposals" -> "UNICODE" )) - - withSQLConf(SQLConf.COLLATION_ENABLED.key -> "false") { - checkError( - exception = intercept[AnalysisException](sql(s"SET COLLATION UNICODE_CI")), - errorClass = "UNSUPPORTED_FEATURE.COLLATION", - parameters = Map.empty - ) - } } test("SPARK-43028: config not found error") { 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 979ff1e24ef5c..c455c2cef15fd 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,7 +113,6 @@ class HiveSessionStateBuilder( PreReadCheck +: TableCapabilityCheck +: CommandCheck +: - CollationCheck +: ViewSyncSchemaToMetaStore +: customCheckRules }