diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index b9a8012275d28..6c7f298199d20 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2315,16 +2315,6 @@ "The value '' in the config \"\" is invalid." ], "subClass" : { - "DEFAULT_COLLATION" : { - "message" : [ - "Cannot resolve the given default collation. Suggested valid collation names: ['']?" - ] - }, - "DEFAULT_COLLATION_NOT_SUPPORTED" : { - "message" : [ - "Setting default session collation other than UTF8_BINARY is currently not supported." - ] - }, "TIME_ZONE" : { "message" : [ "Cannot resolve the given timezone." diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala index 094fd4254e16a..e062ad7781064 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala @@ -23,15 +23,14 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression, Li import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMode} import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataType, ObjectType} +import org.apache.spark.sql.types.{DataType, ObjectType, StringType} private[sql] case class SchemaOfAvro( jsonFormatSchema: String, options: Map[String, String]) extends LeafExpression with RuntimeReplaceable { - override def dataType: DataType = SQLConf.get.defaultStringType + override def dataType: DataType = StringType override def nullable: Boolean = false diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 6cc9ece451c0b..249c5b55b0d71 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -342,8 +342,7 @@ statement ; setResetStatement - : SET COLLATION collationName=identifier #setCollation - | SET ROLE .*? #failSetRole + : SET ROLE .*? #failSetRole | SET TIME ZONE interval #setTimeZone | SET TIME ZONE timezone #setTimeZone | SET TIME ZONE .*? #setTimeZone diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index cb517c689ea16..76449f1704d26 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -20,7 +20,7 @@ import java.util.TimeZone import scala.util.Try -import org.apache.spark.sql.types.{AtomicType, StringType, TimestampType} +import org.apache.spark.sql.types.{AtomicType, TimestampType} import org.apache.spark.util.SparkClassUtils /** @@ -44,7 +44,6 @@ private[sql] trait SqlApiConf { def datetimeJava8ApiEnabled: Boolean def sessionLocalTimeZone: String def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value - def defaultStringType: StringType def stackTracesInDataFrameContext: Int def dataFrameQueryContextEnabled: Boolean def legacyAllowUntypedScalaUDFs: Boolean @@ -61,7 +60,6 @@ private[sql] object SqlApiConf { val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = { SqlApiConfHelper.LOCAL_RELATION_CACHE_THRESHOLD_KEY } - val DEFAULT_COLLATION: String = SqlApiConfHelper.DEFAULT_COLLATION def get: SqlApiConf = SqlApiConfHelper.getConfGetter.get()() @@ -87,7 +85,6 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf { override def datetimeJava8ApiEnabled: Boolean = false override def sessionLocalTimeZone: String = TimeZone.getDefault.getID override def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value = LegacyBehaviorPolicy.CORRECTED - override def defaultStringType: StringType = StringType override def stackTracesInDataFrameContext: Int = 1 override def dataFrameQueryContextEnabled: Boolean = true override def legacyAllowUntypedScalaUDFs: Boolean = false diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala index 486a7dfb58dd0..dace1dbaecfa5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala @@ -32,7 +32,6 @@ private[sql] object SqlApiConfHelper { val CASE_SENSITIVE_KEY: String = "spark.sql.caseSensitive" val SESSION_LOCAL_TIMEZONE_KEY: String = "spark.sql.session.timeZone" val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = "spark.sql.session.localRelationCacheThreshold" - val DEFAULT_COLLATION: String = "spark.sql.session.collation.default" val ARROW_EXECUTION_USE_LARGE_VAR_TYPES = "spark.sql.execution.arrow.useLargeVarTypes" val confGetter: AtomicReference[() => SqlApiConf] = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5477857cd468f..7b23abcf2f503 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -378,7 +378,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ResolveAliases :: ResolveSubquery :: ResolveSubqueryColumnAliases :: - ResolveDefaultStringTypes :: + ResolveDDLCommandStringTypes :: ResolveWindowOrder :: ResolveWindowFrame :: ResolveNaturalAndUsingJoin :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala index 69a4e9fa41785..9fc0cc9700987 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala @@ -246,10 +246,6 @@ object CollationTypeCoercion { if (!expressions.exists(e => SchemaUtils.hasNonUTF8BinaryCollation(e.dataType))) { // if there are no collated types we don't need to do anything return None - } else if (ResolveDefaultStringTypes.needsResolution(expressions)) { - // if any of the strings types are still not resolved - // we need to wait for them to be resolved first - return None } val collationContextWinner = expressions.foldLeft(findCollationContext(expressions.head)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultStringTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDDLCommandStringTypes.scala similarity index 60% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultStringTypes.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDDLCommandStringTypes.scala index 191d03e42b7d2..09ba36f99a270 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDefaultStringTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDDLCommandStringTypes.scala @@ -18,87 +18,57 @@ 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, 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) } private def isCreateOrAlterPlan(plan: LogicalPlan): Boolean = plan match { - case _: V1CreateTablePlan | _: V2CreateTablePlan | _: CreateView | _: AlterViewAs => true + // For CREATE TABLE, only v2 CREATE TABLE command is supported. + // Also, table DEFAULT COLLATION cannot be specified through CREATE TABLE AS SELECT command. + case _: V2CreateTablePlan | _: CreateView | _: AlterViewAs => true case _ => false } @@ -155,22 +125,22 @@ object ResolveDefaultStringTypes extends Rule[LogicalPlan] { dataType.existsRecursively(isDefaultStringType) private def isDefaultStringType(dataType: DataType): Boolean = { + // STRING (without explicit collation) is considered default string type. + // STRING COLLATE (with explicit collation) is not considered + // default string type even when explicit collation is UTF8_BINARY (default collation). dataType match { - case st: StringType => - // should only return true for StringType object and not StringType("UTF8_BINARY") - st.eq(StringType) || st.isInstanceOf[TemporaryStringType] + // should only return true for StringType object and not for StringType("UTF8_BINARY") + case st: StringType => st.eq(StringType) case _ => false } } private def replaceDefaultStringType(dataType: DataType, newType: StringType): DataType = { + // Should replace STRING with the new type. + // Should not replace STRING COLLATE UTF8_BINARY, as that is explicit collation. dataType.transformRecursively { case currentType: StringType if isDefaultStringType(currentType) => - if (currentType == newType) { - TemporaryStringType() - } else { - newType - } + newType } } @@ -186,7 +156,3 @@ object ResolveDefaultStringTypes extends Rule[LogicalPlan] { } } } - -case class TemporaryStringType() extends StringType(1) { - override def toString: String = s"TemporaryStringType($collationId)" -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala index b9e9e49a39647..62f3997491c07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala @@ -29,12 +29,8 @@ import org.apache.spark.sql.catalyst.trees.AlwaysProcess object ResolveInlineTables extends Rule[LogicalPlan] with EvalHelper { override def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsWithPruning(AlwaysProcess.fn, ruleId) { - case table: UnresolvedInlineTable if canResolveTable(table) => + case table: UnresolvedInlineTable if table.expressionsResolved => EvaluateUnresolvedInlineTable.evaluateUnresolvedInlineTable(table) } } - - private def canResolveTable(table: UnresolvedInlineTable): Boolean = { - table.expressionsResolved && !ResolveDefaultStringTypes.needsResolution(table) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index a5b6a17c6ae64..7d4f8c3b2564f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -1467,5 +1467,5 @@ case class MultiCommutativeOp( * Trait for expressions whose data type should be a default string type. */ trait DefaultStringProducingExpression extends Expression { - override def dataType: DataType = SQLConf.get.defaultStringType + override def dataType: DataType = StringType } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index e7cc174f7cf39..1fc8fe8f247b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -79,7 +79,7 @@ case class CreateArray(children: Seq[Expression], useStringTypeWhenEmpty: Boolea private val defaultElementType: DataType = { if (useStringTypeWhenEmpty) { - SQLConf.get.defaultStringType + StringType } else { NullType } @@ -196,7 +196,7 @@ case class CreateMap(children: Seq[Expression], useStringTypeWhenEmpty: Boolean) private val defaultElementType: DataType = { if (useStringTypeWhenEmpty) { - SQLConf.get.defaultStringType + StringType } else { NullType } @@ -354,7 +354,7 @@ case class MapFromArrays(left: Expression, right: Expression) case object NamePlaceholder extends LeafExpression with Unevaluable { override lazy val resolved: Boolean = false override def nullable: Boolean = false - override def dataType: DataType = SQLConf.get.defaultStringType + override def dataType: DataType = StringType override def prettyName: String = "NamePlaceholder" override def toString: String = prettyName } @@ -565,14 +565,14 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E extends TernaryExpression with ExpectsInputTypes { override def nullIntolerant: Boolean = true def this(child: Expression, pairDelim: Expression) = { - this(child, pairDelim, Literal.create(":", SQLConf.get.defaultStringType)) + this(child, pairDelim, Literal.create(":", StringType)) } def this(child: Expression) = { this( child, - Literal.create(",", SQLConf.get.defaultStringType), - Literal.create(":", SQLConf.get.defaultStringType)) + Literal.create(",", StringType), + Literal.create(":", StringType)) } override def stateful: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala index c9d15e1eb2e4d..641f22ba3f786 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala @@ -30,8 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUt import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonGenerator, JacksonParser, JsonInferSchema, JSONOptions} import org.apache.spark.sql.catalyst.util.{ArrayData, FailFastMode, FailureSafeParser, MapData, PermissiveMode} import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType, VariantType} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType, StructField, StructType, VariantType} import org.apache.spark.unsafe.types.{UTF8String, VariantVal} import org.apache.spark.util.Utils @@ -234,7 +233,7 @@ case class SchemaOfJsonEvaluator(options: Map[String, String]) { .getOrElse(ArrayType(StructType(Nil), containsNull = at.containsNull)) case other: DataType => jsonInferSchema.canonicalizeType(other, jsonOptions).getOrElse( - SQLConf.get.defaultStringType) + StringType) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala index 5b17d2029ed1b..5bba89b3dad51 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/maskExpressions.scala @@ -24,9 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.logical.{FunctionSignature, InputParameter} import org.apache.spark.sql.errors.QueryErrorsBase -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation -import org.apache.spark.sql.types.{AbstractDataType, DataType} +import org.apache.spark.sql.types.{AbstractDataType, DataType, StringType} import org.apache.spark.unsafe.types.UTF8String // scalastyle:off line.size.limit @@ -82,13 +81,13 @@ object MaskExpressionBuilder extends ExpressionBuilder { override def functionSignature: Option[FunctionSignature] = { val strArg = InputParameter("str") val upperCharArg = InputParameter("upperChar", - Some(Literal.create(Mask.MASKED_UPPERCASE, SQLConf.get.defaultStringType))) + Some(Literal.create(Mask.MASKED_UPPERCASE, StringType))) val lowerCharArg = InputParameter("lowerChar", - Some(Literal.create(Mask.MASKED_LOWERCASE, SQLConf.get.defaultStringType))) + Some(Literal.create(Mask.MASKED_LOWERCASE, StringType))) val digitCharArg = InputParameter("digitChar", - Some(Literal.create(Mask.MASKED_DIGIT, SQLConf.get.defaultStringType))) + Some(Literal.create(Mask.MASKED_DIGIT, StringType))) val otherCharArg = InputParameter("otherChar", - Some(Literal.create(Mask.MASKED_IGNORE, SQLConf.get.defaultStringType))) + Some(Literal.create(Mask.MASKED_IGNORE, StringType))) val functionSignature: FunctionSignature = FunctionSignature(Seq( strArg, upperCharArg, lowerCharArg, digitCharArg, otherCharArg)) Some(functionSignature) @@ -113,17 +112,17 @@ case class Mask( def this(input: Expression) = this( input, - Literal.create(Mask.MASKED_UPPERCASE, SQLConf.get.defaultStringType), - Literal.create(Mask.MASKED_LOWERCASE, SQLConf.get.defaultStringType), - Literal.create(Mask.MASKED_DIGIT, SQLConf.get.defaultStringType), + Literal.create(Mask.MASKED_UPPERCASE, StringType), + Literal.create(Mask.MASKED_LOWERCASE, StringType), + Literal.create(Mask.MASKED_DIGIT, StringType), Literal.create(Mask.MASKED_IGNORE, input.dataType)) def this(input: Expression, upperChar: Expression) = this( input, upperChar, - Literal.create(Mask.MASKED_LOWERCASE, SQLConf.get.defaultStringType), - Literal.create(Mask.MASKED_DIGIT, SQLConf.get.defaultStringType), + Literal.create(Mask.MASKED_LOWERCASE, StringType), + Literal.create(Mask.MASKED_DIGIT, StringType), Literal.create(Mask.MASKED_IGNORE, input.dataType)) def this(input: Expression, upperChar: Expression, lowerChar: Expression) = @@ -131,7 +130,7 @@ case class Mask( input, upperChar, lowerChar, - Literal.create(Mask.MASKED_DIGIT, SQLConf.get.defaultStringType), + Literal.create(Mask.MASKED_DIGIT, StringType), Literal.create(Mask.MASKED_IGNORE, input.dataType)) def this( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 1788d545160ee..55cc030a99537 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -296,7 +296,7 @@ case class Elt( override def nullable: Boolean = true override def dataType: DataType = - inputExprs.map(_.dataType).headOption.getOrElse(SQLConf.get.defaultStringType) + inputExprs.map(_.dataType).headOption.getOrElse(StringType) override def checkInputDataTypes(): TypeCheckResult = { if (children.size < 2) { @@ -1996,7 +1996,7 @@ object RPadExpressionBuilder extends PadExpressionBuilderBase { case class StringRPad( str: Expression, len: Expression, - pad: Expression = Literal.create(" ", SQLConf.get.defaultStringType)) + pad: Expression = Literal.create(" ", StringType)) extends TernaryExpression with ImplicitCastInputTypes { override def nullIntolerant: Boolean = true override def first: Expression = str @@ -3006,7 +3006,7 @@ object Decode { val input = params.head val other = params.tail val itr = other.iterator - var default: Expression = Literal.create(null, SQLConf.get.defaultStringType) + var default: Expression = Literal.create(null, StringType) val branches = ArrayBuffer.empty[(Expression, Expression)] while (itr.hasNext) { val search = itr.next() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala index 0a72e792a04f6..a609df2204409 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala @@ -446,7 +446,7 @@ case object VariantGet { case Type.BOOLEAN => Literal(v.getBoolean, BooleanType) case Type.LONG => Literal(v.getLong, LongType) case Type.STRING => Literal(UTF8String.fromString(v.getString), - SQLConf.get.defaultStringType) + StringType) case Type.DOUBLE => Literal(v.getDouble, DoubleType) case Type.DECIMAL => val d = Decimal(v.getDecimal) @@ -679,7 +679,7 @@ case class VariantExplode(child: Expression) extends UnaryExpression with Genera override def elementSchema: StructType = { new StructType() .add("pos", IntegerType, nullable = false) - .add("key", SQLConf.get.defaultStringType, nullable = true) + .add("key", StringType, nullable = true) .add("value", VariantType, nullable = false) } } @@ -850,7 +850,7 @@ object SchemaOfVariant { case Type.NULL => NullType case Type.BOOLEAN => BooleanType case Type.LONG => LongType - case Type.STRING => SQLConf.get.defaultStringType + case Type.STRING => StringType case Type.DOUBLE => DoubleType case Type.DECIMAL => val d = Decimal(v.getDecimal) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XmlExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XmlExpressionEvalUtils.scala index 44b98026d62d5..89d7b8d9421a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XmlExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XmlExpressionEvalUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions.xml import org.apache.spark.sql.catalyst.util.GenericArrayData import org.apache.spark.sql.catalyst.xml.XmlInferSchema -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -35,7 +34,7 @@ object XmlExpressionEvalUtils { .map(ArrayType(_, containsNull = at.containsNull)) .getOrElse(ArrayType(StructType(Nil), containsNull = at.containsNull)) case other: DataType => - xmlInferSchema.canonicalizeType(other).getOrElse(SQLConf.get.defaultStringType) + xmlInferSchema.canonicalizeType(other).getOrElse(StringType) } UTF8String.fromString(dataType.sql) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index a524acc19aea8..8f94253c4694f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, instantToMicros, localDateTimeToMicros} import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -155,11 +154,11 @@ case class ReplaceCurrentLike(catalogManager: CatalogManager) extends Rule[Logic plan.transformAllExpressionsWithPruning(_.containsPattern(CURRENT_LIKE)) { case CurrentDatabase() => - Literal.create(currentNamespace, SQLConf.get.defaultStringType) + Literal.create(currentNamespace, StringType) case CurrentCatalog() => - Literal.create(currentCatalog, SQLConf.get.defaultStringType) + Literal.create(currentCatalog, StringType) case CurrentUser() => - Literal.create(currentUser, SQLConf.get.defaultStringType) + Literal.create(currentUser, StringType) } } } 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 05b44f8643698..782c57e12ad69 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 @@ -1214,7 +1214,7 @@ class AstBuilder extends DataTypeAstBuilder } else { // TODO For v2 commands, we will cast the string back to its actual value, // which is a waste and can be improved in the future. - Cast(l, conf.defaultStringType, Some(conf.sessionLocalTimeZone)).eval().toString + Cast(l, StringType, Some(conf.sessionLocalTimeZone)).eval().toString } case other => throw new SparkIllegalArgumentException( @@ -2345,7 +2345,7 @@ class AstBuilder extends DataTypeAstBuilder } val unresolvedTable = UnresolvedInlineTable(aliases, rows.toSeq) - val table = if (canEagerlyEvaluateInlineTable(ctx, unresolvedTable)) { + val table = if (conf.getConf(SQLConf.EAGER_EVAL_OF_UNRESOLVED_INLINE_TABLE_ENABLED)) { EvaluateUnresolvedInlineTable.evaluate(unresolvedTable) } else { unresolvedTable @@ -2353,42 +2353,6 @@ class AstBuilder extends DataTypeAstBuilder table.optionalMap(ctx.tableAlias.strictIdentifier)(aliasPlan) } - /** - * Determines if the inline table can be eagerly evaluated. - */ - private def canEagerlyEvaluateInlineTable( - ctx: InlineTableContext, - table: UnresolvedInlineTable): Boolean = { - if (!conf.getConf(SQLConf.EAGER_EVAL_OF_UNRESOLVED_INLINE_TABLE_ENABLED)) { - return false - } else if (!ResolveDefaultStringTypes.needsResolution(table.expressions)) { - // if there are no strings to be resolved we can always evaluate eagerly - return true - } - - val isSessionCollationSet = conf.defaultStringType != StringType - - // if either of these are true we need to resolve - // the string types first - !isSessionCollationSet && !contextInsideCreate(ctx) - } - - private def contextInsideCreate(ctx: ParserRuleContext): Boolean = { - var currentContext: RuleContext = ctx - - while (currentContext != null) { - if (currentContext.isInstanceOf[CreateTableContext] || - currentContext.isInstanceOf[ReplaceTableContext] || - currentContext.isInstanceOf[CreateViewContext]) { - return true - } - - currentContext = currentContext.parent - } - - false - } - /** * Create an alias (SubqueryAlias) for a join relation. This is practically the same as * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index ae354991ab5f2..1056a30c5f758 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -465,12 +465,6 @@ trait V2CreateTableAsSelectPlan newQuery: LogicalPlan): V2CreateTableAsSelectPlan } -/** - * A trait used for logical plan nodes that create V1 table definitions, - * and so that rules from the catalyst module can identify them. - */ -trait V1CreateTablePlan extends LogicalPlan - /** A trait used for logical plan nodes that create or replace V2 table definitions. */ trait V2CreateTablePlan extends LogicalPlan { def name: LogicalPlan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index bdbf698db2e01..76d36fab2096a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -22,8 +22,7 @@ import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.QueryPlanningTracker -import org.apache.spark.sql.catalyst.rules.RuleExecutor.getForceIterationValue -import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.errors.QueryExecutionErrors @@ -31,27 +30,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils object RuleExecutor { - - /** - * A tag used to explicitly request an additional iteration of the current batch during - * rule execution, even if the query plan remains unchanged. Increment the tag's value - * to enforce another iteration. - */ - private val FORCE_ADDITIONAL_ITERATION = TreeNodeTag[Int]("forceAdditionalIteration") - - /** - * Increments the value of the FORCE_ADDITIONAL_ITERATION tag on the given plan to - * explicitly force another iteration of the current batch during rule execution. - */ - def forceAdditionalIteration(plan: TreeNode[_]): Unit = { - val oldValue = getForceIterationValue(plan) - plan.setTagValue(FORCE_ADDITIONAL_ITERATION, oldValue + 1) - } - - private def getForceIterationValue(plan: TreeNode[_]): Int = { - plan.getTagValue(FORCE_ADDITIONAL_ITERATION).getOrElse(0) - } - protected val queryExecutionMeter = QueryExecutionMetering() /** Dump statistics about time spent running specific rules. */ @@ -325,7 +303,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { continue = false } - if (isFixedPointReached(lastPlan, curPlan)) { + if (curPlan.fastEquals(lastPlan)) { logTrace( s"Fixed point reached for batch ${batch.name} after ${iteration - 1} iterations.") continue = false @@ -339,9 +317,4 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { curPlan } - - private def isFixedPointReached(oldPlan: TreeType, newPlan: TreeType): Boolean = { - oldPlan.fastEquals(newPlan) && - getForceIterationValue(newPlan) <= getForceIterationValue(oldPlan) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/PhysicalDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/PhysicalDataType.scala index 03389f14afa01..a0ed8e5540397 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/PhysicalDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/PhysicalDataType.scala @@ -23,7 +23,6 @@ import scala.reflect.runtime.universe.typeTag import org.apache.spark.sql.catalyst.expressions.{Ascending, BoundReference, InterpretedOrdering, SortOrder} import org.apache.spark.sql.catalyst.util.{ArrayData, CollationFactory, MapData, SQLOrderingUtil} import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteExactNumeric, ByteType, CalendarIntervalType, CharType, DataType, DateType, DayTimeIntervalType, Decimal, DecimalExactNumeric, DecimalType, DoubleExactNumeric, DoubleType, FloatExactNumeric, FloatType, FractionalType, IntegerExactNumeric, IntegerType, IntegralType, LongExactNumeric, LongType, MapType, NullType, NumericType, ShortExactNumeric, ShortType, StringType, StructField, StructType, TimestampNTZType, TimestampType, VarcharType, VariantType, YearMonthIntervalType} import org.apache.spark.unsafe.types.{ByteArray, UTF8String, VariantVal} import org.apache.spark.util.ArrayImplicits._ @@ -41,8 +40,8 @@ object PhysicalDataType { case ShortType => PhysicalShortType case IntegerType => PhysicalIntegerType case LongType => PhysicalLongType - case VarcharType(_) => PhysicalStringType(SqlApiConf.get.defaultStringType.collationId) - case CharType(_) => PhysicalStringType(SqlApiConf.get.defaultStringType.collationId) + case VarcharType(_) => PhysicalStringType(StringType.collationId) + case CharType(_) => PhysicalStringType(StringType.collationId) case s: StringType => PhysicalStringType(s.collationId) case FloatType => PhysicalFloatType case DoubleType => PhysicalDoubleType 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 41bee6ddf6973..6804e46b082c5 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 @@ -43,10 +43,10 @@ import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator import org.apache.spark.sql.catalyst.plans.logical.HintErrorHandler -import org.apache.spark.sql.catalyst.util.{CollationFactory, CollationNames, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} -import org.apache.spark.sql.types.{AtomicType, StringType, TimestampNTZType, TimestampType} +import org.apache.spark.sql.types.{AtomicType, TimestampNTZType, TimestampType} import org.apache.spark.storage.{StorageLevel, StorageLevelMapper} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.{Utils, VersionUtils} @@ -880,40 +880,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - lazy val DEFAULT_COLLATION_ENABLED = - buildConf("spark.sql.sessionDefaultCollation.enabled") - .internal() - .doc("Session default collation 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) - .internal() - .doc("Sets default collation to use for string literals, parameter markers or the string" + - " produced by a builtin function such as to_char or CAST") - .version("4.0.0") - .stringConf - .checkValue( - value => value == CollationNames.UTF8_BINARY || get.getConf(DEFAULT_COLLATION_ENABLED), - errorClass = "DEFAULT_COLLATION_NOT_SUPPORTED", - parameters = _ => Map.empty) - .checkValue( - collationName => { - try { - CollationFactory.fetchCollation(collationName) - true - } catch { - case e: SparkException if e.getCondition == "COLLATION_INVALID_NAME" => false - } - }, - "DEFAULT_COLLATION", - collationName => Map( - "proposals" -> CollationFactory.getClosestSuggestionsOnInvalidName(collationName, 3))) - .createWithDefault("UTF8_BINARY") - val ICU_CASE_MAPPINGS_ENABLED = buildConf("spark.sql.icu.caseMappings.enabled") .doc("When enabled we use the ICU library (instead of the JVM) to implement case mappings" + @@ -5857,14 +5823,6 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def trimCollationEnabled: Boolean = getConf(TRIM_COLLATION_ENABLED) - override def defaultStringType: StringType = { - if (getConf(DEFAULT_COLLATION).toUpperCase(Locale.ROOT) == CollationNames.UTF8_BINARY) { - StringType - } else { - StringType(getConf(DEFAULT_COLLATION)) - } - } - 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/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 10926da52edbf..03ce8882f2fa5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -54,11 +54,6 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case _ if ResolveDefaultStringTypes.needsResolution(plan) => - // if there are still unresolved string types in the plan - // we should not try to resolve it - plan - case AddColumns(ResolvedV1TableIdentifier(ident), cols) => cols.foreach { c => if (c.name.length > 1) { 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 2b7be9b34b9aa..8859b7b421b3c 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 @@ -160,22 +160,6 @@ class SparkSqlAstBuilder extends AstBuilder { ResetCommand(Some(ctx.configKey().getText)) } - /** - * Create a [[SetCommand]] logical plan to set [[SQLConf.DEFAULT_COLLATION]] - * Example SQL : - * {{{ - * SET COLLATION UNICODE; - * }}} - */ - override def visitSetCollation(ctx: SetCollationContext): LogicalPlan = withOrigin(ctx) { - val collationName = ctx.collationName.getText - if (!SQLConf.get.trimCollationEnabled && collationName.toUpperCase().contains("TRIM")) { - throw QueryCompilationErrors.trimCollationNotEnabledError() - } - val key = SQLConf.DEFAULT_COLLATION.key - SetCommand(Some(key -> Some(ctx.identifier.getText.toUpperCase(Locale.ROOT)))) - } - /** * Create a [[SetCommand]] logical plan to set [[SQLConf.SESSION_LOCAL_TIMEZONE]] * Example SQL : diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index e4cecf7d6ead7..f22aa9282ed5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.{Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V1CreateTablePlan} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.command.{DDLUtils, LeafRunnableCommand} @@ -43,7 +43,7 @@ import org.apache.spark.sql.types._ case class CreateTable( tableDesc: CatalogTable, mode: SaveMode, - query: Option[LogicalPlan]) extends LogicalPlan with V1CreateTablePlan { + query: Option[LogicalPlan]) extends LogicalPlan { assert(tableDesc.provider.isDefined, "The table to be created must have a provider.") if (query.isEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala index 8ed9670217859..0d8a61d1932ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql import java.sql.Timestamp -import org.apache.spark.{SparkFunSuite, SparkRuntimeException} +import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis.ExpressionBuilder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.variant.ParseJson -import org.apache.spark.sql.internal.{SqlApiConf, SQLConf} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -686,70 +686,4 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi } } } - - /** - * This test does following: - * 1) Extract all expressions - * 2) Run example queries for different session level default collations - * 3) Check if both expressions throw an exception - * 4) If no exception, check if the result is the same - * 5) Otherwise, check if exceptions are the same - */ - test("SPARK-48280: Expression Walker for SQL query examples") { - val funInfos = spark.sessionState.functionRegistry.listFunction().map { funcId => - spark.sessionState.catalog.lookupFunctionInfo(funcId) - } - - // If expression is expected to return different results, it needs to be skipped - val toSkip = List( - // need to skip as these give timestamp/time related output - "current_timestamp", - "unix_timestamp", - "localtimestamp", - "now", - // need to skip as plans differ in STRING <-> STRING COLLATE UTF8_LCASE - "current_timezone", - "schema_of_variant", - // need to skip as result is expected to differ - "collation", - "contains", - "aes_encrypt", - "translate", - "replace", - "grouping", - "grouping_id", - "reflect", - "try_reflect", - "java_method", - "hash", - "xxhash64", - // need to skip as these are random functions - "rand", - "random", - "randn", - "uuid", - "shuffle", - // other functions which are not yet supported - "to_avro", - "from_avro", - "schema_of_avro", - "to_protobuf", - "from_protobuf" - ) - - for (funInfo <- funInfos.filter(f => !toSkip.contains(f.getName))) { - for (query <- "> .*;".r.findAllIn(funInfo.getExamples).map(s => s.substring(2))) { - try { - val resultUTF8 = sql(query).collect() - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UTF8_LCASE") { - val resultUTF8Lcase = sql(query).collect() - assert(resultUTF8 === resultUTF8Lcase) - } - } catch { - case e: SparkRuntimeException => assert(e.getCondition == "USER_RAISED_EXCEPTION") - case other: Throwable => throw new Exception(s"Query $query failed", other) - } - } - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala deleted file mode 100644 index 384411a0fd342..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala +++ /dev/null @@ -1,3525 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import java.sql.{Date, Timestamp} -import java.text.SimpleDateFormat - -import scala.collection.immutable.Seq - -import org.apache.spark.{SparkConf, SparkException, SparkIllegalArgumentException, SparkRuntimeException} -import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, InternalRow} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.Mode -import org.apache.spark.sql.catalyst.util.CollationFactory -import org.apache.spark.sql.internal.{SqlApiConf, SQLConf} -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.util.collection.OpenHashMap - -// scalastyle:off nonascii -class CollationSQLExpressionsSuite - extends QueryTest - with SharedSparkSession - with ExpressionEvalHelper { - - private val testSuppCollations = - Seq( - "UTF8_BINARY", - "UTF8_BINARY_RTRIM", - "UTF8_LCASE", - "UTF8_LCASE_RTRIM", - "UNICODE", - "UNICODE_RTRIM", - "UNICODE_CI", - "UNICODE_CI_RTRIM") - private val testAdditionalCollations = Seq("UNICODE", - "SR", "SR_RTRIM", "SR_CI", "SR_AI", "SR_CI_AI") - private val fullyQualifiedPrefix = s"${CollationFactory.CATALOG}.${CollationFactory.SCHEMA}." - - test("Support Md5 hash expression with collation") { - case class Md5TestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - Md5TestCase("Spark", "UTF8_BINARY", "8cde774d6f7333752ed72cacddb05126"), - Md5TestCase("Spark", "UTF8_BINARY_RTRIM", "8cde774d6f7333752ed72cacddb05126"), - Md5TestCase("Spark", "UTF8_LCASE", "8cde774d6f7333752ed72cacddb05126"), - Md5TestCase("Spark", "UTF8_LCASE_RTRIM", "8cde774d6f7333752ed72cacddb05126"), - Md5TestCase("SQL", "UNICODE", "9778840a0100cb30c982876741b0b5a2"), - Md5TestCase("SQL", "UNICODE_RTRIM", "9778840a0100cb30c982876741b0b5a2"), - Md5TestCase("SQL", "UNICODE_CI", "9778840a0100cb30c982876741b0b5a2"), - Md5TestCase("SQL", "UNICODE_CI_RTRIM", "9778840a0100cb30c982876741b0b5a2"), - Md5TestCase("SQL", "SR_CI_AI", "9778840a0100cb30c982876741b0b5a2") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select md5('${t.input}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support Sha2 hash expression with collation") { - case class Sha2TestCase( - input: String, - collationName: String, - bitLength: Int, - result: String - ) - - val testCases = Seq( - Sha2TestCase("Spark", "UTF8_BINARY", 256, - "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), - Sha2TestCase("Spark", "UTF8_BINARY_RTRIM", 256, - "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), - Sha2TestCase("Spark", "UTF8_LCASE", 256, - "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), - Sha2TestCase("Spark", "UTF8_LCASE_RTRIM", 256, - "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), - Sha2TestCase("SQL", "UNICODE", 256, - "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"), - Sha2TestCase("SQL", "UNICODE_RTRIM", 256, - "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"), - Sha2TestCase("SQL", "UNICODE_CI", 256, - "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"), - Sha2TestCase("SQL", "UNICODE_CI_RTRIM", 256, - "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"), - Sha2TestCase("SQL", "SR_AI", 256, - "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select sha2('${t.input}', ${t.bitLength}) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support Sha1 hash expression with collation") { - case class Sha1TestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - Sha1TestCase("Spark", "UTF8_BINARY", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), - Sha1TestCase("Spark", "UTF8_BINARY_RTRIM", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), - Sha1TestCase("Spark", "UTF8_LCASE", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), - Sha1TestCase("Spark", "UTF8_LCASE_RTRIM", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), - Sha1TestCase("SQL", "UNICODE", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"), - Sha1TestCase("SQL", "UNICODE_RTRIM", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"), - Sha1TestCase("SQL", "UNICODE_CI", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"), - Sha1TestCase("SQL", "UNICODE_CI_RTRIM", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"), - Sha1TestCase("Spark", "SR_CI", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select sha1('${t.input}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support Crc32 hash expression with collation") { - case class Crc321TestCase( - input: String, - collationName: String, - result: Int - ) - - val testCases = Seq( - Crc321TestCase("Spark", "UTF8_BINARY", 1557323817), - Crc321TestCase("Spark", "UTF8_BINARY_RTRIM", 1557323817), - Crc321TestCase("Spark", "UTF8_LCASE", 1557323817), - Crc321TestCase("Spark", "UTF8_LCASE_RTRIM", 1557323817), - Crc321TestCase("SQL", "UNICODE", 1299261525), - Crc321TestCase("SQL", "UNICODE_RTRIM", 1299261525), - Crc321TestCase("SQL", "UNICODE_CI", 1299261525), - Crc321TestCase("SQL", "UNICODE_CI_RTRIM", 1299261525) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select crc32('${t.input}') - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - } - }) - } - - test("Support Murmur3Hash hash expression with collation") { - case class Murmur3HashTestCase( - input: String, - collationName: String, - result: Int - ) - - val testCases = Seq( - Murmur3HashTestCase("Spark", "UTF8_BINARY", 228093765), - Murmur3HashTestCase("Spark ", "UTF8_BINARY_RTRIM", 1779328737), - Murmur3HashTestCase("Spark", "UTF8_LCASE", -1928694360), - Murmur3HashTestCase("Spark ", "UTF8_LCASE_RTRIM", -1928694360), - Murmur3HashTestCase("SQL", "UNICODE", 1483684981), - Murmur3HashTestCase("SQL ", "UNICODE_RTRIM", 1483684981), - Murmur3HashTestCase("SQL", "UNICODE_CI", 279787709), - Murmur3HashTestCase("SQL ", "UNICODE_CI_RTRIM", 279787709) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select hash('${t.input}') - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - } - }) - } - - test("Support XxHash64 hash expression with collation") { - case class XxHash64TestCase( - input: String, - collationName: String, - result: Long - ) - - val testCases = Seq( - XxHash64TestCase("Spark", "UTF8_BINARY", -4294468057691064905L), - XxHash64TestCase("Spark ", "UTF8_BINARY_RTRIM", 6480371823304753502L), - XxHash64TestCase("Spark", "UTF8_LCASE", -3142112654825786434L), - XxHash64TestCase("Spark ", "UTF8_LCASE_RTRIM", -3142112654825786434L), - XxHash64TestCase("SQL", "UNICODE", 7549349329256749019L), - XxHash64TestCase("SQL ", "UNICODE_RTRIM", 7549349329256749019L), - XxHash64TestCase("SQL", "UNICODE_CI", -3010409544364398863L), - XxHash64TestCase("SQL ", "UNICODE_CI_RTRIM", -3010409544364398863L) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select xxhash64('${t.input}') - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - } - }) - } - - test("Support UrlEncode hash expression with collation") { - case class UrlEncodeTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - UrlEncodeTestCase("https://spark.apache.org", "UTF8_BINARY", - "https%3A%2F%2Fspark.apache.org"), - UrlEncodeTestCase("https://spark.apache.org", "UTF8_BINARY_RTRIM", - "https%3A%2F%2Fspark.apache.org"), - UrlEncodeTestCase("https://spark.apache.org", "UTF8_LCASE", - "https%3A%2F%2Fspark.apache.org"), - UrlEncodeTestCase("https://spark.apache.org", "UTF8_LCASE_RTRIM", - "https%3A%2F%2Fspark.apache.org"), - UrlEncodeTestCase("https://spark.apache.org", "UNICODE", - "https%3A%2F%2Fspark.apache.org"), - UrlEncodeTestCase("https://spark.apache.org", "UNICODE_RTRIM", - "https%3A%2F%2Fspark.apache.org"), - UrlEncodeTestCase("https://spark.apache.org", "UNICODE_CI", - "https%3A%2F%2Fspark.apache.org"), - UrlEncodeTestCase("https://spark.apache.org", "UNICODE_CI_RTRIM", - "https%3A%2F%2Fspark.apache.org") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select url_encode('${t.input}') - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support UrlDecode hash expression with collation") { - case class UrlDecodeTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_BINARY", - "https://spark.apache.org"), - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_BINARY_RTRIM", - "https://spark.apache.org"), - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_LCASE", - "https://spark.apache.org"), - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_LCASE_RTRIM", - "https://spark.apache.org"), - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE", - "https://spark.apache.org"), - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE_RTRIM", - "https://spark.apache.org"), - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE_CI", - "https://spark.apache.org"), - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE_CI_RTRIM", - "https://spark.apache.org") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select url_decode('${t.input}') - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support ParseUrl hash expression with collation") { - case class ParseUrlTestCase( - input: String, - collationName: String, - path: String, - result: String - ) - - val testCases = Seq( - ParseUrlTestCase("http://spark.apache.org/path?query=1", "UTF8_BINARY", "HOST", - "spark.apache.org"), - ParseUrlTestCase("http://spark.apache.org/path?query=1", "UTF8_BINARY_RTRIM", "HOST", - "spark.apache.org"), - ParseUrlTestCase("http://spark.apache.org/path?query=2", "UTF8_LCASE", "PATH", - "/path"), - ParseUrlTestCase("http://spark.apache.org/path?query=2", "UTF8_LCASE_RTRIM", "PATH", - "/path"), - ParseUrlTestCase("http://spark.apache.org/path?query=3", "UNICODE", "QUERY", - "query=3"), - ParseUrlTestCase("http://spark.apache.org/path?query=3", "UNICODE_RTRIM", "QUERY", - "query=3"), - ParseUrlTestCase("http://spark.apache.org/path?query=4", "UNICODE_CI", "PROTOCOL", - "http"), - ParseUrlTestCase("http://spark.apache.org/path?query=4", "UNICODE_CI_RTRIM", "PROTOCOL", - "http") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select parse_url('${t.input}', '${t.path}') - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support CsvToStructs csv expression with collation") { - case class CsvToStructsTestCase( - input: String, - collationName: String, - schema: String, - options: String, - result: Row, - structFields: Seq[StructField] - ) - - val testCases = Seq( - CsvToStructsTestCase("1", "UTF8_BINARY", "'a INT'", "", - Row(1), Seq( - StructField("a", IntegerType, nullable = true) - )), - CsvToStructsTestCase("1", "UTF8_BINARY_RTRIM", "'a INT'", "", - Row(1), Seq( - StructField("a", IntegerType, nullable = true) - )), - CsvToStructsTestCase("true, 0.8", "UTF8_LCASE", "'A BOOLEAN, B DOUBLE'", "", - Row(true, 0.8), Seq( - StructField("A", BooleanType, nullable = true), - StructField("B", DoubleType, nullable = true) - )), - CsvToStructsTestCase("true, 0.8", "UTF8_LCASE_RTRIM", "'A BOOLEAN, B DOUBLE'", "", - Row(true, 0.8), Seq( - StructField("A", BooleanType, nullable = true), - StructField("B", DoubleType, nullable = true) - )), - CsvToStructsTestCase("\"Spark\"", "UNICODE", "'a STRING'", "", - Row("Spark"), Seq( - StructField("a", StringType, nullable = true) - )), - CsvToStructsTestCase("\"Spark\"", "UTF8_BINARY", "'a STRING COLLATE UNICODE'", "", - Row("Spark"), Seq( - StructField("a", StringType("UNICODE"), nullable = true) - )), - CsvToStructsTestCase("\"Spark\"", "UNICODE_RTRIM", "'a STRING COLLATE UNICODE_RTRIM'", "", - Row("Spark"), Seq( - StructField("a", StringType("UNICODE_RTRIM"), nullable = true) - )), - CsvToStructsTestCase("26/08/2015", "UTF8_BINARY", "'time Timestamp'", - ", map('timestampFormat', 'dd/MM/yyyy')", Row( - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S").parse("2015-08-26 00:00:00.0") - ), Seq( - StructField("time", TimestampType, nullable = true) - )) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select from_csv('${t.input}', ${t.schema} ${t.options}) - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - val queryResult = testQuery.collect().head - checkAnswer(testQuery, Row(t.result)) - val dataType = StructType(t.structFields) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support SchemaOfCsv csv expression with collation") { - case class SchemaOfCsvTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - SchemaOfCsvTestCase("1", "UTF8_BINARY", "STRUCT<_c0: INT>"), - SchemaOfCsvTestCase("1", "UTF8_BINARY_RTRIM", "STRUCT<_c0: INT>"), - SchemaOfCsvTestCase("true,0.8", "UTF8_LCASE", - "STRUCT<_c0: BOOLEAN, _c1: DOUBLE>"), - SchemaOfCsvTestCase("true,0.8", "UTF8_LCASE_RTRIM", - "STRUCT<_c0: BOOLEAN, _c1: DOUBLE>"), - SchemaOfCsvTestCase("2015-08-26", "UNICODE", "STRUCT<_c0: DATE>"), - SchemaOfCsvTestCase("2015-08-26", "UNICODE_RTRIM", "STRUCT<_c0: DATE>"), - SchemaOfCsvTestCase("abc", "UNICODE_CI", - "STRUCT<_c0: STRING>"), - SchemaOfCsvTestCase("abc", "UNICODE_CI_RTRIM", - "STRUCT<_c0: STRING>") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select schema_of_csv('${t.input}') - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support StructsToCsv csv expression with collation") { - case class StructsToCsvTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - StructsToCsvTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY", "1,2"), - StructsToCsvTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY_RTRIM", "1,2"), - StructsToCsvTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_LCASE", "true,2.0"), - StructsToCsvTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_LCASE_RTRIM", "true,2.0"), - StructsToCsvTestCase("named_struct()", "UNICODE", null), - StructsToCsvTestCase("named_struct()", "UNICODE_RTRIM", null), - StructsToCsvTestCase("named_struct('time', to_timestamp('2015-08-26'))", "UNICODE_CI", - "2015-08-26T00:00:00.000-07:00"), - StructsToCsvTestCase("named_struct('time', to_timestamp('2015-08-26'))", "UNICODE_CI_RTRIM", - "2015-08-26T00:00:00.000-07:00") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select to_csv(${t.input}) - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Conv expression with collation") { - // Supported collations - case class ConvTestCase( - num: String, - from_base: String, - to_base: String, - collationName: String, - result: String) - - val testCases = Seq( - ConvTestCase("100", "2", "10", "UTF8_BINARY", "4"), - ConvTestCase("100", "2", "10", "UTF8_BINARY_RTRIM", "4"), - ConvTestCase("100", "2", "10", "UTF8_LCASE", "4"), - ConvTestCase("100", "2", "10", "UTF8_LCASE_RTRIM", "4"), - ConvTestCase("100", "2", "10", "UNICODE", "4"), - ConvTestCase("100", "2", "10", "UNICODE_RTRIM", "4"), - ConvTestCase("100", "2", "10", "UNICODE_CI", "4"), - ConvTestCase("100", "2", "10", "UNICODE_CI_RTRIM", "4") - ) - testCases.foreach(t => { - val query = - s""" - |select conv(collate('${t.num}', '${t.collationName}'), ${t.from_base}, ${t.to_base}) - |""".stripMargin - // Result & data type - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collationName))) - }) - } - - test("Bin expression with collation") { - // Supported collations - case class BinTestCase( - num: String, - collationName: String, - result: String) - - val testCases = Seq( - BinTestCase("13", "UTF8_BINARY", "1101"), - BinTestCase("13", "UTF8_BINARY_RTRIM", "1101"), - BinTestCase("13", "UTF8_LCASE", "1101"), - BinTestCase("13", "UTF8_LCASE_RTRIM", "1101"), - BinTestCase("13", "UNICODE", "1101"), - BinTestCase("13", "UNICODE_RTRIM", "1101"), - BinTestCase("13", "UNICODE_CI", "1101"), - BinTestCase("13", "UNICODE_CI_RTRIM", "1101") - ) - testCases.foreach(t => { - val query = - s""" - |select bin(${t.num}) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collationName))) - } - }) - } - - test("Hex with non-string input expression with collation") { - case class HexTestCase( - num: String, - collationName: String, - result: String) - - val testCases = Seq( - HexTestCase("13", "UTF8_BINARY", "D"), - HexTestCase("13", "UTF8_BINARY_RTRIM", "D"), - HexTestCase("13", "UTF8_LCASE", "D"), - HexTestCase("13", "UTF8_LCASE_RTRIM", "D"), - HexTestCase("13", "UNICODE", "D"), - HexTestCase("13", "UNICODE_RTRIM", "D"), - HexTestCase("13", "UNICODE_CI", "D"), - HexTestCase("13", "UNICODE_CI_RTRIM", "D") - ) - testCases.foreach(t => { - val query = - s""" - |select hex(${t.num}) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collationName))) - } - }) - } - - test("Hex with string input expression with collation") { - case class HexTestCase( - num: String, - collationName: String, - result: String) - - val testCases = Seq( - HexTestCase("Spark SQL", "UTF8_BINARY", "537061726B2053514C"), - HexTestCase("Spark SQL", "UTF8_BINARY_RTRIM", "537061726B2053514C"), - HexTestCase("Spark SQL", "UTF8_LCASE", "537061726B2053514C"), - HexTestCase("Spark SQL", "UTF8_LCASE_RTRIM", "537061726B2053514C"), - HexTestCase("Spark SQL", "UNICODE", "537061726B2053514C"), - HexTestCase("Spark SQL", "UNICODE_RTRIM", "537061726B2053514C"), - HexTestCase("Spark SQL", "UNICODE_CI", "537061726B2053514C"), - HexTestCase("Spark SQL", "UNICODE_CI_RTRIM", "537061726B2053514C"), - HexTestCase("Spark SQL", "DE_CI_AI", "537061726B2053514C"), - HexTestCase("Spark SQL", "DE_CI_AI_RTRIM", "537061726B2053514C") - ) - testCases.foreach(t => { - val query = - s""" - |select hex(collate('${t.num}', '${t.collationName}')) - |""".stripMargin - // Result & data type - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collationName))) - }) - } - - test("UnHex expression with collation") { - case class UnHexTestCase( - num: String, - collationName: String, - result: String) - - val testCases = Seq( - UnHexTestCase("537061726B2053514C", "UTF8_BINARY", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "UTF8_BINARY_RTRIM", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "UTF8_LCASE", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "UTF8_LCASE_RTRIM", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "UNICODE", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "UNICODE_RTRIM", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "UNICODE_CI", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "UNICODE_CI_RTRIM", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "DE", "Spark SQL") - ) - testCases.foreach(t => { - val query = - s""" - |select decode(unhex(collate('${t.num}', '${t.collationName}')), 'UTF-8') - |""".stripMargin - // Result & data type - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType("UTF8_BINARY"))) - }) - } - - test("Support XPath expressions with collation") { - case class XPathTestCase( - xml: String, - xpath: String, - functionName: String, - collationName: String, - result: Any, - resultType: DataType - ) - - val testCases = Seq( - XPathTestCase("1", "a/b", - "xpath_boolean", "UTF8_BINARY", true, BooleanType), - XPathTestCase("12", "sum(A/B)", - "xpath_short", "UTF8_BINARY", 3, ShortType), - XPathTestCase("1", "a/b", - "xpath_boolean", "UTF8_BINARY_RTRIM", true, BooleanType), - XPathTestCase("12", "sum(A/B)", - "xpath_short", "UTF8_BINARY_RTRIM", 3, ShortType), - XPathTestCase("34", "sum(a/b)", - "xpath_int", "UTF8_LCASE", 7, IntegerType), - XPathTestCase("56", "sum(A/B)", - "xpath_long", "UTF8_LCASE", 11, LongType), - XPathTestCase("34", "sum(a/b)", - "xpath_int", "UTF8_LCASE_RTRIM", 7, IntegerType), - XPathTestCase("56", "sum(A/B)", - "xpath_long", "UTF8_LCASE_RTRIM", 11, LongType), - XPathTestCase("78", "sum(a/b)", - "xpath_float", "UNICODE", 15.0, FloatType), - XPathTestCase("90", "sum(A/B)", - "xpath_double", "UNICODE", 9.0, DoubleType), - XPathTestCase("78", "sum(a/b)", - "xpath_float", "UNICODE_RTRIM", 15.0, FloatType), - XPathTestCase("90", "sum(A/B)", - "xpath_double", "UNICODE_RTRIM", 9.0, DoubleType), - XPathTestCase("bcc", "a/c", - "xpath_string", "UNICODE_CI", "cc", StringType("UNICODE_CI")), - XPathTestCase("bcc ", "a/c", - "xpath_string", "UNICODE_CI_RTRIM", "cc ", StringType("UNICODE_CI_RTRIM")), - XPathTestCase("b1b2b3c1c2", "a/b/text()", - "xpath", "UNICODE_CI", Array("b1", "b2", "b3"), ArrayType(StringType("UNICODE_CI"))) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select ${t.functionName}('${t.xml}', '${t.xpath}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - assert(testQuery.schema.fields.head.dataType.sameType(t.resultType)) - } - }) - } - - test("Support StringSpace expression with collation") { - case class StringSpaceTestCase( - input: Int, - collationName: String, - result: String - ) - - val testCases = Seq( - StringSpaceTestCase(1, "UTF8_BINARY", " "), - StringSpaceTestCase(1, "UTF8_BINARY_RTRIM", " "), - StringSpaceTestCase(2, "UTF8_LCASE", " "), - StringSpaceTestCase(2, "UTF8_LCASE_RTRIM", " "), - StringSpaceTestCase(3, "UNICODE", " "), - StringSpaceTestCase(3, "UNICODE_RTRIM", " "), - StringSpaceTestCase(4, "UNICODE_CI", " "), - StringSpaceTestCase(4, "UNICODE_CI_RTRIM", " "), - StringSpaceTestCase(5, "AF_CI_AI", " "), - StringSpaceTestCase(5, "AF_CI_AI_RTRIM", " ") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select space(${t.input}) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support ToNumber & TryToNumber expressions with collation") { - case class ToNumberTestCase( - input: String, - collationName: String, - format: String, - result: Any, - resultType: DataType - ) - - val testCases = Seq( - ToNumberTestCase("123", "UTF8_BINARY", "999", 123, DecimalType(3, 0)), - ToNumberTestCase("123", "UTF8_BINARY_RTRIM", "999", 123, DecimalType(3, 0)), - ToNumberTestCase("1", "UTF8_LCASE", "0.00", 1.00, DecimalType(3, 2)), - ToNumberTestCase("1", "UTF8_LCASE_RTRIM", "0.00", 1.00, DecimalType(3, 2)), - ToNumberTestCase("99,999", "UNICODE", "99,999", 99999, DecimalType(5, 0)), - ToNumberTestCase("99,999", "UNICODE_RTRIM", "99,999", 99999, DecimalType(5, 0)), - ToNumberTestCase("$14.99", "UNICODE_CI", "$99.99", 14.99, DecimalType(4, 2)), - ToNumberTestCase("$14.99", "UNICODE_CI_RTRIM", "$99.99", 14.99, DecimalType(4, 2)) - ) - - // Supported collations (ToNumber) - testCases.foreach(t => { - val query = - s""" - |select to_number('${t.input}', '${t.format}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - assert(testQuery.schema.fields.head.dataType.sameType(t.resultType)) - } - }) - - // Supported collations (TryToNumber) - testCases.foreach(t => { - val query = - s""" - |select try_to_number('${t.input}', '${t.format}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - assert(testQuery.schema.fields.head.dataType.sameType(t.resultType)) - } - }) - } - - test("Handle invalid number for ToNumber variant expression with collation") { - // to_number should throw an exception if the conversion fails - val number = "xx" - val query = s"SELECT to_number('$number', '999');" - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkError( - exception = intercept[SparkIllegalArgumentException] { - val testQuery = sql(query) - testQuery.collect() - }, - condition = "INVALID_FORMAT.MISMATCH_INPUT", - parameters = Map("inputType" -> "\"STRING\"", "input" -> "xx", "format" -> "999") - ) - } - } - - test("Handle invalid number for TryToNumber variant expression with collation") { - // try_to_number shouldn't throw an exception if the conversion fails - val number = "xx" - val query = s"SELECT try_to_number('$number', '999');" - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - val testQuery = sql(query) - checkAnswer(testQuery, Row(null)) - } - } - - test("Support ToChar expression with collation") { - case class ToCharTestCase( - input: Int, - collationName: String, - format: String, - result: String - ) - - val testCases = Seq( - ToCharTestCase(12, "UTF8_BINARY", "999", " 12"), - ToCharTestCase(12, "UTF8_BINARY_RTRIM", "999", " 12"), - ToCharTestCase(34, "UTF8_LCASE", "000D00", "034.00"), - ToCharTestCase(34, "UTF8_LCASE_RTRIM", "000D00", "034.00"), - ToCharTestCase(56, "UNICODE", "$99.99", "$56.00"), - ToCharTestCase(56, "UNICODE_RTRIM", "$99.99", "$56.00"), - ToCharTestCase(78, "UNICODE_CI", "99D9S", "78.0+"), - ToCharTestCase(78, "UNICODE_CI_RTRIM", "99D9S", "78.0+") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select to_char(${t.input}, '${t.format}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support GetJsonObject json expression with collation") { - case class GetJsonObjectTestCase( - input: String, - path: String, - collationName: String, - result: String - ) - - val testCases = Seq( - GetJsonObjectTestCase("{\"a\":\"b\"}", "$.a", "UTF8_BINARY", "b"), - GetJsonObjectTestCase("{\"a\":\"b\"}", "$.a", "UTF8_BINARY_RTRIM", "b"), - GetJsonObjectTestCase("{\"A\":\"1\"}", "$.A", "UTF8_LCASE", "1"), - GetJsonObjectTestCase("{\"A\":\"1\"}", "$.A", "UTF8_LCASE_RTRIM", "1"), - GetJsonObjectTestCase("{\"x\":true}", "$.x", "UNICODE", "true"), - GetJsonObjectTestCase("{\"x\":true}", "$.x", "UNICODE_RTRIM", "true"), - GetJsonObjectTestCase("{\"X\":1}", "$.X", "UNICODE_CI", "1"), - GetJsonObjectTestCase("{\"X\":1}", "$.X", "UNICODE_CI_RTRIM", "1") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT get_json_object('${t.input}', '${t.path}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support JsonTuple json expression with collation") { - case class JsonTupleTestCase( - input: String, - names: String, - collationName: String, - result: Row - ) - - val testCases = Seq( - JsonTupleTestCase("{\"a\":1, \"b\":2}", "'a', 'b'", "UTF8_BINARY", - Row("1", "2")), - JsonTupleTestCase("{\"a\":1, \"b\":2}", "'a', 'b'", "UTF8_BINARY_RTRIM", - Row("1", "2")), - JsonTupleTestCase("{\"A\":\"3\", \"B\":\"4\"}", "'A', 'B'", "UTF8_LCASE", - Row("3", "4")), - JsonTupleTestCase("{\"A\":\"3\", \"B\":\"4\"}", "'A', 'B'", "UTF8_LCASE_RTRIM", - Row("3", "4")), - JsonTupleTestCase("{\"x\":true, \"y\":false}", "'x', 'y'", "UNICODE", - Row("true", "false")), - JsonTupleTestCase("{\"x\":true, \"y\":false}", "'x', 'y'", "UNICODE_RTRIM", - Row("true", "false")), - JsonTupleTestCase("{\"X\":null, \"Y\":null}", "'X', 'Y'", "UNICODE_CI", - Row(null, null)) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT json_tuple('${t.input}', ${t.names}) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, t.result) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support JsonToStructs json expression with collation") { - case class JsonToStructsTestCase( - input: String, - schema: String, - collationName: String, - result: Row - ) - - val testCases = Seq( - JsonToStructsTestCase("{\"a\":1, \"b\":2.0}", "a INT, b DOUBLE", - "UTF8_BINARY", Row(Row(1, 2.0))), - JsonToStructsTestCase("{\"a\":1, \"b\":2.0}", "a INT, b DOUBLE", - "UTF8_BINARY_RTRIM", Row(Row(1, 2.0))), - JsonToStructsTestCase("{\"A\":\"3\", \"B\":4}", "A STRING COLLATE UTF8_LCASE, B INT", - "UTF8_LCASE", Row(Row("3", 4))), - JsonToStructsTestCase("{\"A\":\"3\", \"B\":4}", "A STRING COLLATE UTF8_LCASE, B INT", - "UTF8_LCASE_RTRIM", Row(Row("3", 4))), - JsonToStructsTestCase("{\"x\":true, \"y\":null}", "x BOOLEAN, y VOID", - "UNICODE", Row(Row(true, null))), - JsonToStructsTestCase("{\"x\":true, \"y\":null}", "x BOOLEAN, y VOID", - "UNICODE_RTRIM", Row(Row(true, null))), - JsonToStructsTestCase("{\"X\":null, \"Y\":false}", "X VOID, Y BOOLEAN", - "UNICODE_CI", Row(Row(null, false))), - JsonToStructsTestCase("{\"X\":null, \"Y\":false}", "X VOID, Y BOOLEAN", - "UNICODE_CI_RTRIM", Row(Row(null, false))) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT from_json('${t.input}', '${t.schema}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, t.result) - val dataType = StructType.fromDDL(t.schema) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support StructsToJson json expression with collation") { - case class StructsToJsonTestCase( - struct: String, - collationName: String, - result: Row - ) - - val testCases = Seq( - StructsToJsonTestCase("named_struct('a', 1, 'b', 2)", - "UTF8_BINARY", Row("{\"a\":1,\"b\":2}")), - StructsToJsonTestCase("named_struct('a', 1, 'b', 2)", - "UTF8_BINARY_RTRIM", Row("{\"a\":1,\"b\":2}")), - StructsToJsonTestCase("array(named_struct('a', 1, 'b', 2))", - "UTF8_LCASE", Row("[{\"a\":1,\"b\":2}]")), - StructsToJsonTestCase("array(named_struct('a', 1, 'b', 2))", - "UTF8_LCASE_RTRIM", Row("[{\"a\":1,\"b\":2}]")), - StructsToJsonTestCase("map('a', named_struct('b', 1))", - "UNICODE", Row("{\"a\":{\"b\":1}}")), - StructsToJsonTestCase("map('a', named_struct('b', 1))", - "UNICODE_RTRIM", Row("{\"a\":{\"b\":1}}")), - StructsToJsonTestCase("array(map('a', 1))", - "UNICODE_CI", Row("[{\"a\":1}]")), - StructsToJsonTestCase("array(map('a', 1))", - "UNICODE_CI_RTRIM", Row("[{\"a\":1}]")) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT to_json(${t.struct}) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, t.result) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support LengthOfJsonArray json expression with collation") { - case class LengthOfJsonArrayTestCase( - input: String, - collationName: String, - result: Row - ) - - val testCases = Seq( - LengthOfJsonArrayTestCase("'[1,2,3,4]'", "UTF8_BINARY", Row(4)), - LengthOfJsonArrayTestCase("'[1,2,3,4]'", "UTF8_BINARY_RTRIM", Row(4)), - LengthOfJsonArrayTestCase("'[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]'", "UTF8_LCASE", Row(5)), - LengthOfJsonArrayTestCase("'[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]'", "UTF8_LCASE_RTRIM", Row(5)), - LengthOfJsonArrayTestCase("'[1,2'", "UNICODE", Row(null)), - LengthOfJsonArrayTestCase("'[1,2'", "UNICODE_RTRIM", Row(null)), - LengthOfJsonArrayTestCase("'['", "UNICODE_CI", Row(null)), - LengthOfJsonArrayTestCase("'['", "UNICODE_CI_RTRIM", Row(null)) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT json_array_length(${t.input}) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, t.result) - assert(testQuery.schema.fields.head.dataType.sameType(IntegerType)) - } - }) - } - - test("Support JsonObjectKeys json expression with collation") { - case class JsonObjectKeysJsonArrayTestCase( - input: String, - collationName: String, - result: Row - ) - - val testCases = Seq( - JsonObjectKeysJsonArrayTestCase("{}", "UTF8_BINARY", - Row(Seq())), - JsonObjectKeysJsonArrayTestCase("{}", "UTF8_BINARY_RTRIM", - Row(Seq())), - JsonObjectKeysJsonArrayTestCase("{\"k\":", "UTF8_LCASE", - Row(null)), - JsonObjectKeysJsonArrayTestCase("{\"k\":", "UTF8_LCASE_RTRIM", - Row(null)), - JsonObjectKeysJsonArrayTestCase("{\"k1\": \"v1\"}", "UNICODE", - Row(Seq("k1"))), - JsonObjectKeysJsonArrayTestCase("{\"k1\": \"v1\"}", "UNICODE_RTRIM", - Row(Seq("k1"))), - JsonObjectKeysJsonArrayTestCase("{\"k1\":1,\"k2\":{\"k3\":3, \"k4\":4}}", "UNICODE_CI", - Row(Seq("k1", "k2"))), - JsonObjectKeysJsonArrayTestCase("{\"k1\":1,\"k2\":{\"k3\":3, \"k4\":4}}", "UNICODE_CI_RTRIM", - Row(Seq("k1", "k2"))) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT json_object_keys('${t.input}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, t.result) - val dataType = ArrayType(StringType(t.collationName)) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support SchemaOfJson json expression with collation") { - case class SchemaOfJsonTestCase( - input: String, - collationName: String, - result: Row - ) - - val testCases = Seq( - SchemaOfJsonTestCase("'[{\"col\":0}]'", - "UTF8_BINARY", Row("ARRAY>")), - SchemaOfJsonTestCase("'[{\"col\":0}]'", - "UTF8_BINARY_RTRIM", Row("ARRAY>")), - SchemaOfJsonTestCase("'[{\"col\":01}]', map('allowNumericLeadingZeros', 'true')", - "UTF8_LCASE", Row("ARRAY>")), - SchemaOfJsonTestCase("'[{\"col\":01}]', map('allowNumericLeadingZeros', 'true')", - "UTF8_LCASE_RTRIM", Row("ARRAY>")), - SchemaOfJsonTestCase("'[]'", - "UNICODE", Row("ARRAY")), - SchemaOfJsonTestCase("'[]'", - "UNICODE_RTRIM", Row("ARRAY")), - SchemaOfJsonTestCase("''", - "UNICODE_CI", Row("STRING")), - SchemaOfJsonTestCase("''", - "UNICODE_CI_RTRIM", Row("STRING")) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT schema_of_json(${t.input}) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, t.result) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support `StringToMap` expression with collation") { - case class StringToMapTestCase[R]( - text: String, - pairDelim: String, - keyValueDelim: String, - collation: String, - result: R) - val testCases = Seq( - StringToMapTestCase("a:1,b:2,c:3", ",", ":", "UTF8_BINARY", - Map("a" -> "1", "b" -> "2", "c" -> "3")), - StringToMapTestCase("A-1xB-2xC-3", "X", "-", "UTF8_LCASE", - Map("A" -> "1", "B" -> "2", "C" -> "3")), - StringToMapTestCase("1:ax2:bx3:c", "x", ":", "UNICODE", - Map("1" -> "a", "2" -> "b", "3" -> "c")), - StringToMapTestCase("1/AX2/BX3/C", "x", "/", "UNICODE_CI", - Map("1" -> "A", "2" -> "B", "3" -> "C")), - StringToMapTestCase("1:cx2:čx3:ć", "x", ":", "SR_CI_AI", - Map("1" -> "c", "2" -> "č", "3" -> "ć")), - StringToMapTestCase("c:1,č:2,ć:3", ",", ":", "SR_CI", - Map("c" -> "1", "č" -> "2", "ć" -> "3")) - ) - val unsupportedTestCases = Seq( - StringToMapTestCase("a:1,b:2,c:3", "?", "?", "UNICODE_AI", null)) - testCases.foreach(t => { - // Unit test. - val text = Literal.create(t.text, StringType(t.collation)) - val pairDelim = Literal.create(t.pairDelim, StringType(t.collation)) - val keyValueDelim = Literal.create(t.keyValueDelim, StringType(t.collation)) - checkEvaluation(StringToMap(text, pairDelim, keyValueDelim), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"SELECT str_to_map('${t.text}', '${t.pairDelim}', '${t.keyValueDelim}')" - checkAnswer(sql(query), Row(t.result)) - val dataType = MapType(StringType(t.collation), StringType(t.collation), true) - assert(sql(query).schema.fields.head.dataType.sameType(dataType)) - } - }) - // Test unsupported collation. - unsupportedTestCases.foreach(t => { - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = - s"select str_to_map('${t.text}', '${t.pairDelim}', " + - s"'${t.keyValueDelim}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> ("\"str_to_map('a:1,b:2,c:3' collate " + s"${t.collation}, " + - "'?' collate " + s"${t.collation}, '?' collate ${t.collation})" + "\""), - "paramIndex" -> "first", - "inputSql" -> ("\"'a:1,b:2,c:3' collate " + s"${t.collation}" + "\""), - "inputType" -> ("\"STRING COLLATE " + s"${t.collation}" + "\""), - "requiredType" -> "\"STRING\""), - context = ExpectedContext( - fragment = "str_to_map('a:1,b:2,c:3', '?', '?')", - start = 7, - stop = 41)) - } - }) - } - - test("Support RaiseError misc expression with collation") { - // Supported collations - case class RaiseErrorTestCase(errorMessage: String, collationName: String) - val testCases = Seq( - RaiseErrorTestCase("custom error message 1", "UTF8_BINARY"), - RaiseErrorTestCase("custom error message 1", "UTF8_BINARY_RTRIM"), - RaiseErrorTestCase("custom error message 2", "UTF8_LCASE"), - RaiseErrorTestCase("custom error message 2", "UTF8_LCASE_RTRIM"), - RaiseErrorTestCase("custom error message 3", "UNICODE"), - RaiseErrorTestCase("custom error message 3", "UNICODE_RTRIM"), - RaiseErrorTestCase("custom error message 4", "UNICODE_CI"), - RaiseErrorTestCase("custom error message 4", "UNICODE_CI_RTRIM") - ) - testCases.foreach(t => { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val query = s"SELECT raise_error('${t.errorMessage}')" - // Result & data type - checkError( - exception = intercept[SparkRuntimeException] { - sql(query).collect() - }, - condition = "USER_RAISED_EXCEPTION", - parameters = Map("errorMessage" -> t.errorMessage) - ) - } - }) - } - - test("Support CurrentDatabase/Catalog/User expressions with collation") { - // Supported collations - Seq( - "UTF8_LCASE", - "UTF8_LCASE_RTRIM", - "UNICODE", - "UNICODE_RTRIM", - "UNICODE_CI", - "SR_CI_AI").foreach(collationName => - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val queryDatabase = sql("SELECT current_schema()") - val queryCatalog = sql("SELECT current_catalog()") - val queryUser = sql("SELECT current_user()") - // Data type - val dataType = StringType(collationName) - assert(queryDatabase.schema.fields.head.dataType.sameType(dataType)) - assert(queryCatalog.schema.fields.head.dataType.sameType(dataType)) - assert(queryUser.schema.fields.head.dataType.sameType(dataType)) - } - ) - } - - test("Support Uuid misc expression with collation") { - // Supported collations - Seq( - "UTF8_LCASE", - "UTF8_LCASE_RTRIM", - "UNICODE", - "UNICODE_RTRIM", - "UNICODE_CI", - "UNICODE_CI_RTRIM", - "NO_CI_AI").foreach(collationName => - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val query = s"SELECT uuid()" - // Result & data type - val testQuery = sql(query) - val queryResult = testQuery.collect().head.getString(0) - val uuidFormat = "^[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}$" - assert(queryResult.matches(uuidFormat)) - val dataType = StringType(collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - ) - } - - test("Support SparkVersion misc expression with collation") { - // Supported collations - Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI", "DE").foreach(collationName => - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val query = s"SELECT version()" - // Result & data type - val testQuery = sql(query) - val queryResult = testQuery.collect().head.getString(0) - val versionFormat = "^[0-9]\\.[0-9]\\.[0-9] [0-9a-f]{40}$" - assert(queryResult.matches(versionFormat)) - val dataType = StringType(collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - ) - } - - test("Support TypeOf misc expression with collation") { - // Supported collations - case class TypeOfTestCase(input: String, collationName: String, result: String) - val testCases = Seq( - TypeOfTestCase("1", "UTF8_BINARY", "int"), - TypeOfTestCase("\"A\"", "UTF8_LCASE", "string collate UTF8_LCASE"), - TypeOfTestCase("array(1)", "UNICODE", "array"), - TypeOfTestCase("null", "UNICODE_CI", "void") - ) - testCases.foreach(t => { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val query = s"SELECT typeof(${t.input})" - // Result & data type - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support AesEncrypt misc expression with collation") { - // Supported collations - case class AesEncryptTestCase( - input: String, - collationName: String, - params: String, - result: String - ) - val testCases = Seq( - AesEncryptTestCase("Spark", "UTF8_BINARY", "'1234567890abcdef', 'ECB'", - "8DE7DB79A23F3E8ED530994DDEA98913"), - AesEncryptTestCase("Spark", "UTF8_LCASE", "'1234567890abcdef', 'ECB', 'DEFAULT', ''", - "8DE7DB79A23F3E8ED530994DDEA98913"), - AesEncryptTestCase("Spark", "UNICODE", "'1234567890abcdef', 'GCM', 'DEFAULT', " + - "unhex('000000000000000000000000')", - "00000000000000000000000046596B2DE09C729FE48A0F81A00A4E7101DABEB61D"), - AesEncryptTestCase("Spark", "UNICODE_CI", "'1234567890abcdef', 'CBC', 'DEFAULT', " + - "unhex('00000000000000000000000000000000')", - "000000000000000000000000000000008DE7DB79A23F3E8ED530994DDEA98913") - ) - testCases.foreach(t => { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val query = s"SELECT hex(aes_encrypt('${t.input}', ${t.params}))" - // Result & data type - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support AesDecrypt misc expression with collation") { - // Supported collations - case class AesDecryptTestCase( - input: String, - collationName: String, - params: String, - result: String - ) - val testCases = Seq( - AesDecryptTestCase("8DE7DB79A23F3E8ED530994DDEA98913", - "UTF8_BINARY", "'1234567890abcdef', 'ECB'", "Spark"), - AesDecryptTestCase("8DE7DB79A23F3E8ED530994DDEA98913", - "UTF8_LCASE", "'1234567890abcdef', 'ECB', 'DEFAULT', ''", "Spark"), - AesDecryptTestCase("00000000000000000000000046596B2DE09C729FE48A0F81A00A4E7101DABEB61D", - "UNICODE", "'1234567890abcdef', 'GCM', 'DEFAULT'", "Spark"), - AesDecryptTestCase("000000000000000000000000000000008DE7DB79A23F3E8ED530994DDEA98913", - "UNICODE_CI", "'1234567890abcdef', 'CBC', 'DEFAULT'", "Spark") - ) - testCases.foreach(t => { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val query = s"SELECT aes_decrypt(unhex('${t.input}'), ${t.params})" - // Result & data type - val testQuery = sql(query) - checkAnswer(testQuery, sql(s"SELECT to_binary('${t.result}', 'utf-8')")) - assert(testQuery.schema.fields.head.dataType.sameType(BinaryType)) - } - }) - } - - test("Support Mask expression with collation") { - // Supported collations - case class MaskTestCase[R](i: String, u: String, l: String, d: String, o: String, c: String, - result: R) - val testCases = Seq( - MaskTestCase("ab-CD-12-@$", null, null, null, null, "UTF8_BINARY", "ab-CD-12-@$"), - MaskTestCase("ab-CD-12-@$", "X", null, null, null, "UTF8_LCASE", "ab-XX-12-@$"), - MaskTestCase("ab-CD-12-@$", "X", "x", null, null, "UNICODE", "xx-XX-12-@$"), - MaskTestCase("ab-CD-12-@$", "X", "x", "0", "#", "UNICODE_CI", "xx#XX#00###") - ) - testCases.foreach(t => { - def col(s: String): String = if (s == null) "null" else s"collate('$s', '${t.c}')" - val query = s"SELECT mask(${col(t.i)}, ${col(t.u)}, ${col(t.l)}, ${col(t.d)}, ${col(t.o)})" - // Result & data type - var result = sql(query) - checkAnswer(result, Row(t.result)) - assert(result.schema.fields.head.dataType.sameType(StringType(t.c))) - }) - // Implicit casting - val testCasting = Seq( - MaskTestCase("ab-CD-12-@$", "X", "x", "0", "#", "UNICODE_CI", "xx#XX#00###") - ) - testCasting.foreach(t => { - def col(s: String): String = if (s == null) "null" else s"collate('$s', '${t.c}')" - def str(s: String): String = if (s == null) "null" else s"'$s'" - val query1 = s"SELECT mask(${col(t.i)}, ${str(t.u)}, ${str(t.l)}, ${str(t.d)}, ${str(t.o)})" - val query2 = s"SELECT mask(${str(t.i)}, ${col(t.u)}, ${str(t.l)}, ${str(t.d)}, ${str(t.o)})" - val query3 = s"SELECT mask(${str(t.i)}, ${str(t.u)}, ${col(t.l)}, ${str(t.d)}, ${str(t.o)})" - val query4 = s"SELECT mask(${str(t.i)}, ${str(t.u)}, ${str(t.l)}, ${col(t.d)}, ${str(t.o)})" - val query5 = s"SELECT mask(${str(t.i)}, ${str(t.u)}, ${str(t.l)}, ${str(t.d)}, ${col(t.o)})" - for (q <- Seq(query1, query2, query3, query4, query5)) { - val result = sql(q) - checkAnswer(result, Row(t.result)) - assert(result.schema.fields.head.dataType.sameType(StringType(t.c))) - } - }) - // Collation mismatch - checkError( - exception = intercept[AnalysisException] { - sql("SELECT mask(collate('ab-CD-12-@$','UNICODE'),collate('X','UNICODE_CI'),'x','0','#')") - }, - condition = "COLLATION_MISMATCH.EXPLICIT", - parameters = Map( - "explicitTypes" -> """"STRING COLLATE UNICODE", "STRING COLLATE UNICODE_CI"""" - ) - ) - } - - test("Support XmlToStructs xml expression with collation") { - case class XmlToStructsTestCase( - input: String, - collationName: String, - schema: String, - options: String, - result: Row, - structFields: Seq[StructField] - ) - - val testCases = Seq( - XmlToStructsTestCase("

1

", "UTF8_BINARY", "'a INT'", "", - Row(1), Seq( - StructField("a", IntegerType, nullable = true) - )), - XmlToStructsTestCase("

1

", "UTF8_BINARY_RTRIM", "'a INT'", "", - Row(1), Seq( - StructField("a", IntegerType, nullable = true) - )), - XmlToStructsTestCase("

true0.8

", "UTF8_LCASE", - "'A BOOLEAN, B DOUBLE'", "", Row(true, 0.8), Seq( - StructField("A", BooleanType, nullable = true), - StructField("B", DoubleType, nullable = true) - )), - XmlToStructsTestCase("

true0.8

", "UTF8_LCASE_RTRIM", - "'A BOOLEAN, B DOUBLE'", "", Row(true, 0.8), Seq( - StructField("A", BooleanType, nullable = true), - StructField("B", DoubleType, nullable = true) - )), - XmlToStructsTestCase("

Spark

", "UNICODE", "'s STRING'", "", - Row("Spark"), Seq( - StructField("s", StringType, nullable = true) - )), - XmlToStructsTestCase("

Spark

", "UTF8_BINARY", "'s STRING COLLATE UNICODE'", "", - Row("Spark"), Seq( - StructField("s", StringType("UNICODE"), nullable = true) - )), - XmlToStructsTestCase("

Spark

", "UNICODE_RTRIM", - "'s STRING COLLATE UNICODE_RTRIM'", "", - Row("Spark"), Seq( - StructField("s", StringType("UNICODE_RTRIM"), nullable = true) - )), - XmlToStructsTestCase("

", "UNICODE_CI", "'time Timestamp'", - ", map('timestampFormat', 'dd/MM/yyyy')", Row( - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.S").parse("2015-08-26 00:00:00.0") - ), Seq( - StructField("time", TimestampType, nullable = true) - )) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select from_xml('${t.input}', ${t.schema} ${t.options}) - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StructType(t.structFields) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support SchemaOfXml xml expression with collation") { - case class SchemaOfXmlTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - SchemaOfXmlTestCase("

1

", "UTF8_BINARY", "STRUCT"), - SchemaOfXmlTestCase("

1

", "UTF8_BINARY_RTRIM", "STRUCT"), - SchemaOfXmlTestCase("

true0.8

", "UTF8_LCASE", - "STRUCT"), - SchemaOfXmlTestCase("

true0.8

", "UTF8_LCASE_RTRIM", - "STRUCT"), - SchemaOfXmlTestCase("

", "UNICODE", "STRUCT<>"), - SchemaOfXmlTestCase("

", "UNICODE_RTRIM", "STRUCT<>"), - SchemaOfXmlTestCase("

123

", "UNICODE_CI", - "STRUCT>"), - SchemaOfXmlTestCase("

123

", "UNICODE_CI_RTRIM", - "STRUCT>") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select schema_of_xml('${t.input}') - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support StructsToXml xml expression with collation") { - case class StructsToXmlTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - StructsToXmlTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY", - s""" - | 1 - | 2 - |""".stripMargin), - StructsToXmlTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY_RTRIM", - s""" - | 1 - | 2 - |""".stripMargin), - StructsToXmlTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_LCASE", - s""" - | true - | 2.0 - |""".stripMargin), - StructsToXmlTestCase("named_struct('A', 'aa', 'B', 'bb')", "UTF8_LCASE", - s""" - | aa - | bb - |""".stripMargin), - StructsToXmlTestCase("named_struct('A', 'aa', 'B', 'bb')", "UTF8_LCASE_RTRIM", - s""" - | aa - | bb - |""".stripMargin), - StructsToXmlTestCase("named_struct('A', 'aa', 'B', 'bb')", "UTF8_BINARY", - s""" - | aa - | bb - |""".stripMargin), - StructsToXmlTestCase("named_struct()", "UNICODE", - ""), - StructsToXmlTestCase("named_struct()", "UNICODE_RTRIM", - ""), - StructsToXmlTestCase("named_struct('time', to_timestamp('2015-08-26'))", "UNICODE_CI", - s""" - | - |""".stripMargin) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |select to_xml(${t.input}) - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - val dataType = StringType(t.collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("Support ParseJson & TryParseJson variant expressions with collation") { - case class ParseJsonTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - ParseJsonTestCase("{\"a\":1,\"b\":2}", "UTF8_BINARY", "{\"a\":1,\"b\":2}"), - ParseJsonTestCase("{\"a\":1,\"b\":2}", "UTF8_BINARY_RTRIM", "{\"a\":1,\"b\":2}"), - ParseJsonTestCase("{\"A\":3,\"B\":4}", "UTF8_LCASE", "{\"A\":3,\"B\":4}"), - ParseJsonTestCase("{\"A\":3,\"B\":4}", "UTF8_LCASE_RTRIM", "{\"A\":3,\"B\":4}"), - ParseJsonTestCase("{\"c\":5,\"d\":6}", "UNICODE", "{\"c\":5,\"d\":6}"), - ParseJsonTestCase("{\"c\":5,\"d\":6}", "UNICODE_RTRIM", "{\"c\":5,\"d\":6}"), - ParseJsonTestCase("{\"C\":7,\"D\":8}", "UNICODE_CI", "{\"C\":7,\"D\":8}"), - ParseJsonTestCase("{\"C\":7,\"D\":8}", "UNICODE_CI_RTRIM", "{\"C\":7,\"D\":8}") - ) - - // Supported collations (ParseJson) - testCases.foreach(t => { - val query = - s""" - |SELECT parse_json('${t.input}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - val testResult = testQuery.collect().map(_.toString()).mkString("") - assert(testResult === "[" + t.result + "]") // can't use checkAnswer for Variant - assert(testQuery.schema.fields.head.dataType.sameType(VariantType)) - } - }) - - // Supported collations (TryParseJson) - testCases.foreach(t => { - val query = - s""" - |SELECT try_parse_json('${t.input}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - val testResult = testQuery.collect().map(_.toString()).mkString("") - assert(testResult === "[" + t.result + "]") // can't use checkAnswer for Variant - assert(testQuery.schema.fields.head.dataType.sameType(VariantType)) - } - }) - } - - test("Handle invalid JSON for ParseJson variant expression with collation") { - // parse_json should throw an exception when the string is not valid JSON value - val json = "{\"a\":1," - val query = s"SELECT parse_json('$json');" - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkError( - exception = intercept[SparkException] { - val testQuery = sql(query) - testQuery.collect() - }, - condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", - parameters = Map("badRecord" -> "{\"a\":1,", "failFastMode" -> "FAILFAST") - ) - } - } - - test("Handle invalid JSON for TryParseJson variant expression with collation") { - // try_parse_json shouldn't throw an exception when the string is not valid JSON value - val json = "{\"a\":1,]" - val query = s"SELECT try_parse_json('$json');" - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - val testQuery = sql(query) - val testResult = testQuery.collect().map(_.toString()).mkString("") - assert(testResult === s"[null]") - } - } - - test("Support IsVariantNull variant expressions with collation") { - case class IsVariantNullTestCase( - input: String, - collationName: String, - result: Boolean - ) - - val testCases = Seq( - IsVariantNullTestCase("'null'", "UTF8_BINARY", result = true), - IsVariantNullTestCase("'null'", "UTF8_BINARY_RTRIM", result = true), - IsVariantNullTestCase("'\"null\"'", "UTF8_LCASE", result = false), - IsVariantNullTestCase("'\"null\"'", "UTF8_LCASE_RTRIM", result = false), - IsVariantNullTestCase("'13'", "UNICODE", result = false), - IsVariantNullTestCase("'13'", "UNICODE_RTRIM", result = false), - IsVariantNullTestCase("null", "UNICODE_CI", result = false), - IsVariantNullTestCase("null", "UNICODE_CI_RTRIM", result = false) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT is_variant_null(parse_json(${t.input})) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - } - }) - } - - test("Support VariantGet & TryVariantGet variant expressions with collation") { - case class VariantGetTestCase( - input: String, - path: String, - variantType: String, - collationName: String, - result: Any, - resultType: DataType - ) - - val testCases = Seq( - VariantGetTestCase("{\"a\": 1}", "$.a", "int", "UTF8_BINARY", 1, IntegerType), - VariantGetTestCase("{\"a\": 1}", "$.a", "int", "UTF8_BINARY_RTRIM", 1, IntegerType), - VariantGetTestCase("{\"a\": 1}", "$.b", "int", "UTF8_LCASE", null, IntegerType), - VariantGetTestCase("[1, \"2\"]", "$[1]", "string", "UNICODE", "2", - StringType), - VariantGetTestCase("[1, \"2\"]", "$[1]", "string collate unicode", "UTF8_BINARY", "2", - StringType("UNICODE")), - VariantGetTestCase("[1, \"2\"]", "$[2]", "string", "UNICODE_CI", null, - StringType), - VariantGetTestCase("[1, \"2\"]", "$[2]", "string collate unicode_CI", "UTF8_BINARY", null, - StringType("UNICODE_CI")) - ) - - // Supported collations (VariantGet) - testCases.foreach(t => { - val query = - s""" - |SELECT variant_get(parse_json('${t.input}'), '${t.path}', '${t.variantType}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - val testResult = testQuery.collect().map(_.toString()).mkString("") - assert(testResult === "[" + t.result + "]") // can't use checkAnswer for Variant - assert(testQuery.schema.fields.head.dataType.sameType(t.resultType)) - } - }) - - // Supported collations (TryVariantGet) - testCases.foreach(t => { - val query = - s""" - |SELECT try_variant_get(parse_json('${t.input}'), '${t.path}', '${t.variantType}') - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - val testResult = testQuery.collect().map(_.toString()).mkString("") - assert(testResult === "[" + t.result + "]") // can't use checkAnswer for Variant - assert(testQuery.schema.fields.head.dataType.sameType(t.resultType)) - } - }) - } - - test("Handle invalid JSON for VariantGet variant expression with collation") { - // variant_get should throw an exception if the cast fails - val json = "[1, \"Spark\"]" - val query = s"SELECT variant_get(parse_json('$json'), '$$[1]', 'int');" - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkError( - exception = intercept[SparkRuntimeException] { - val testQuery = sql(query) - testQuery.collect() - }, - condition = "INVALID_VARIANT_CAST", - parameters = Map("value" -> "\"Spark\"", "dataType" -> "\"INT\"") - ) - } - } - - test("Handle invalid JSON for TryVariantGet variant expression with collation") { - // try_variant_get shouldn't throw an exception if the cast fails - val json = "[1, \"Spark\"]" - val query = s"SELECT try_variant_get(parse_json('$json'), '$$[1]', 'int');" - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - val testQuery = sql(query) - val testResult = testQuery.collect().map(_.toString()).mkString("") - assert(testResult === s"[null]") - } - } - - test("Support VariantExplode variant expressions with collation") { - case class VariantExplodeTestCase( - input: String, - collationName: String, - result: String, - resultType: Seq[StructField] - ) - - val testCases = Seq( - VariantExplodeTestCase("[\"hello\", \"world\"]", "UTF8_BINARY", - Row(0, "null", "\"hello\"").toString() + Row(1, "null", "\"world\"").toString(), - Seq[StructField]( - StructField("pos", IntegerType, nullable = false), - StructField("key", StringType("UTF8_BINARY")), - StructField("value", VariantType, nullable = false) - ) - ), - VariantExplodeTestCase("[\"hello\", \"world\"]", "UTF8_BINARY_RTRIM", - Row(0, "null", "\"hello\"").toString() + Row(1, "null", "\"world\"").toString(), - Seq[StructField]( - StructField("pos", IntegerType, nullable = false), - StructField("key", StringType("UTF8_BINARY_RTRIM")), - StructField("value", VariantType, nullable = false) - ) - ), - VariantExplodeTestCase("[\"Spark\", \"SQL\"]", "UTF8_LCASE", - Row(0, "null", "\"Spark\"").toString() + Row(1, "null", "\"SQL\"").toString(), - Seq[StructField]( - StructField("pos", IntegerType, nullable = false), - StructField("key", StringType("UTF8_LCASE")), - StructField("value", VariantType, nullable = false) - ) - ), - VariantExplodeTestCase("[\"Spark\", \"SQL\"]", "UTF8_LCASE_RTRIM", - Row(0, "null", "\"Spark\"").toString() + Row(1, "null", "\"SQL\"").toString(), - Seq[StructField]( - StructField("pos", IntegerType, nullable = false), - StructField("key", StringType("UTF8_LCASE_RTRIM")), - StructField("value", VariantType, nullable = false) - ) - ), - VariantExplodeTestCase("{\"a\": true, \"b\": 3.14}", "UNICODE", - Row(0, "a", "true").toString() + Row(1, "b", "3.14").toString(), - Seq[StructField]( - StructField("pos", IntegerType, nullable = false), - StructField("key", StringType("UNICODE")), - StructField("value", VariantType, nullable = false) - ) - ), - VariantExplodeTestCase("{\"a\": true, \"b\": 3.14}", "UNICODE_RTRIM", - Row(0, "a", "true").toString() + Row(1, "b", "3.14").toString(), - Seq[StructField]( - StructField("pos", IntegerType, nullable = false), - StructField("key", StringType("UNICODE_RTRIM")), - StructField("value", VariantType, nullable = false) - ) - ), - VariantExplodeTestCase("{\"A\": 9.99, \"B\": false}", "UNICODE_CI", - Row(0, "A", "9.99").toString() + Row(1, "B", "false").toString(), - Seq[StructField]( - StructField("pos", IntegerType, nullable = false), - StructField("key", StringType("UNICODE_CI")), - StructField("value", VariantType, nullable = false) - ) - ) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT * from variant_explode(parse_json('${t.input}')) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - val testResult = testQuery.collect().map(_.toString()).mkString("") - assert(testResult === t.result) // can't use checkAnswer for Variant - assert(testQuery.schema.fields.sameElements(t.resultType)) - } - }) - } - - test("Support SchemaOfVariant variant expressions with collation") { - case class SchemaOfVariantTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - SchemaOfVariantTestCase("null", "UTF8_BINARY", "VOID"), - SchemaOfVariantTestCase("null", "UTF8_BINARY_RTRIM", "VOID"), - SchemaOfVariantTestCase("[]", "UTF8_LCASE", "ARRAY"), - SchemaOfVariantTestCase("[]", "UTF8_LCASE_RTRIM", "ARRAY"), - SchemaOfVariantTestCase("[{\"a\":true,\"b\":0}]", "UNICODE", - "ARRAY>"), - SchemaOfVariantTestCase("[{\"a\":true,\"b\":0}]", "UNICODE_RTRIM", - "ARRAY>"), - SchemaOfVariantTestCase("[{\"A\":\"x\",\"B\":-1.00}]", "UNICODE_CI", - "ARRAY>"), - SchemaOfVariantTestCase("[{\"A\":\"x\",\"B\":-1.00}]", "UNICODE_CI_RTRIM", - "ARRAY>") - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT schema_of_variant(parse_json('${t.input}')) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - assert(testQuery.schema.fields.head.dataType.sameType(StringType(t.collationName))) - } - }) - } - - test("Support SchemaOfVariantAgg variant expressions with collation") { - case class SchemaOfVariantAggTestCase( - input: String, - collationName: String, - result: String - ) - - val testCases = Seq( - SchemaOfVariantAggTestCase("('1'), ('2'), ('3')", "UTF8_BINARY", "BIGINT"), - SchemaOfVariantAggTestCase("('1'), ('2'), ('3')", "UTF8_BINARY_RTRIM", "BIGINT"), - SchemaOfVariantAggTestCase("('true'), ('false'), ('true')", "UTF8_LCASE", "BOOLEAN"), - SchemaOfVariantAggTestCase("('true'), ('false'), ('true')", "UTF8_LCASE_RTRIM", "BOOLEAN"), - SchemaOfVariantAggTestCase("('{\"a\": 1}'), ('{\"b\": true}'), ('{\"c\": 1.23}')", - "UNICODE", "OBJECT"), - SchemaOfVariantAggTestCase("('{\"a\": 1}'), ('{\"b\": true}'), ('{\"c\": 1.23}')", - "UNICODE_RTRIM", "OBJECT"), - SchemaOfVariantAggTestCase("('{\"A\": \"x\"}'), ('{\"B\": 9.99}'), ('{\"C\": 0}')", - "UNICODE_CI", "OBJECT"), - SchemaOfVariantAggTestCase("('{\"A\": \"x\"}'), ('{\"B\": 9.99}'), ('{\"C\": 0}')", - "UNICODE_CI_RTRIM", "OBJECT" - ) - ) - - // Supported collations - testCases.foreach(t => { - val query = - s""" - |SELECT schema_of_variant_agg(parse_json(j)) FROM VALUES ${t.input} AS tab(j) - |""".stripMargin - // Result & data type - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row(t.result)) - assert(testQuery.schema.fields.head.dataType.sameType(StringType(t.collationName))) - } - }) - } - - test("Support InputFileName expression with collation") { - // Supported collations - Seq( - "UTF8_BINARY", - "UTF8_BINARY_RTRIM", - "UTF8_LCASE", - "UTF8_LCASE_RTRIM", - "UNICODE", - "UNICODE_RTRIM", - "UNICODE_CI", - "UNICODE_CI_RTRIM", - "MT_CI_AI").foreach(collationName => { - val query = - s""" - |select input_file_name() - |""".stripMargin - // Result - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val testQuery = sql(query) - checkAnswer(testQuery, Row("")) - val dataType = StringType(collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("DateFormat expression with collation") { - case class DateFormatTestCase[R](date: String, format: String, collation: String, result: R) - val testCases = Seq( - DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UTF8_BINARY", "2021-01-01"), - DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UTF8_BINARY_RTRIM", "2021-01-01"), - DateFormatTestCase("2021-01-01", "yyyy-dd", "UTF8_LCASE", "2021-01"), - DateFormatTestCase("2021-01-01", "yyyy-dd", "UTF8_LCASE_RTRIM", "2021-01"), - DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UNICODE", "2021-01-01"), - DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UNICODE_RTRIM", "2021-01-01"), - DateFormatTestCase("2021-01-01", "yyyy", "UNICODE_CI", "2021"), - DateFormatTestCase("2021-01-01", "yyyy", "UNICODE_CI_RTRIM", "2021") - ) - - for { - collateDate <- Seq(true, false) - collateFormat <- Seq(true, false) - } { - testCases.foreach(t => { - val dateArg = if (collateDate) s"collate('${t.date}', '${t.collation}')" else s"'${t.date}'" - val formatArg = - if (collateFormat) { - s"collate('${t.format}', '${t.collation}')" - } else { - s"'${t.format}'" - } - - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collation) { - val query = s"SELECT date_format(${dateArg}, ${formatArg})" - // Result & data type - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - } - - test("Support mode for string expression with collation - Basic Test") { - Seq( - "utf8_binary", - "utf8_binary_rtrim", - "UTF8_LCASE", - "UTF8_LCASE_RTRIM", - "unicode_ci", - "unicode_ci_rtrim", - "unicode", - "unicode_rtrim", - "NL_AI").foreach { collationId => - val query = s"SELECT mode(collate('abc', '${collationId}'))" - checkAnswer(sql(query), Row("abc")) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(collationId))) - } - } - - test("Support mode for string expression with collation - Advanced Test") { - case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) - val testCases = Seq( - ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("UTF8_LCASE_RTRIM", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("SR", Map("c" -> 3L, "č" -> 2L, "Č" -> 2L), "c") - ) - testCases.foreach(t => { - val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => - (0L to numRepeats).map(_ => s"('$elt')").mkString(",") - }.mkString(",") - - val tableName = s"t_${t.collationId}_mode" - withTable(s"${tableName}") { - sql(s"CREATE TABLE ${tableName}(i STRING) USING parquet") - sql(s"INSERT INTO ${tableName} VALUES " + valuesToAdd) - val query = s"SELECT mode(collate(i, '${t.collationId}')) FROM ${tableName}" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collationId))) - - } - }) - } - - test("Support Mode.eval(buffer)") { - case class UTF8StringModeTestCase[R]( - collationId: String, - bufferValues: Map[UTF8String, Long], - result: R) - - val bufferValuesUTF8String = Map( - UTF8String.fromString("a") -> 5L, - UTF8String.fromString("b") -> 4L, - UTF8String.fromString("B") -> 3L, - UTF8String.fromString("d") -> 2L, - UTF8String.fromString("e") -> 1L) - - val testCasesUTF8String = Seq( - UTF8StringModeTestCase("utf8_binary", bufferValuesUTF8String, "a"), - UTF8StringModeTestCase("utf8_binary_rtrim", bufferValuesUTF8String, "a"), - UTF8StringModeTestCase("UTF8_LCASE", bufferValuesUTF8String, "b"), - UTF8StringModeTestCase("UTF8_LCASE_RTRIM", bufferValuesUTF8String, "b"), - UTF8StringModeTestCase("unicode_ci", bufferValuesUTF8String, "b"), - UTF8StringModeTestCase("unicode_ci_rtrim", bufferValuesUTF8String, "b"), - UTF8StringModeTestCase("unicode", bufferValuesUTF8String, "a"), - UTF8StringModeTestCase("unicode_rtrim", bufferValuesUTF8String, "a") - ) - - testCasesUTF8String.foreach ( t => { - val buffer = new OpenHashMap[AnyRef, Long](5) - val myMode = Mode(child = Literal.create("some_column_name", StringType(t.collationId))) - t.bufferValues.foreach { case (k, v) => buffer.update(k, v) } - assert(myMode.eval(buffer).toString.toLowerCase() == t.result.toLowerCase()) - }) - } - - test("Support Mode.eval(buffer) with complex types") { - case class UTF8StringModeTestCase[R]( - collationId: String, - bufferValues: Map[InternalRow, Long], - result: R) - - val bufferValuesUTF8String: Map[Any, Long] = Map( - UTF8String.fromString("a") -> 5L, - UTF8String.fromString("b") -> 4L, - UTF8String.fromString("B") -> 3L, - UTF8String.fromString("d") -> 2L, - UTF8String.fromString("e") -> 1L) - - val bufferValuesComplex = bufferValuesUTF8String.map{ - case (k, v) => (InternalRow.fromSeq(Seq(k, k, k)), v) - } - val testCasesUTF8String = Seq( - UTF8StringModeTestCase("utf8_binary", bufferValuesComplex, "[a,a,a]"), - UTF8StringModeTestCase("utf8_binary_rtrim", bufferValuesComplex, "[a,a,a]"), - UTF8StringModeTestCase("UTF8_LCASE", bufferValuesComplex, "[b,b,b]"), - UTF8StringModeTestCase("UTF8_LCASE_rtrim", bufferValuesComplex, "[b,b,b]"), - UTF8StringModeTestCase("unicode_ci", bufferValuesComplex, "[b,b,b]"), - UTF8StringModeTestCase("unicode_ci_rtrim", bufferValuesComplex, "[b,b,b]"), - UTF8StringModeTestCase("unicode", bufferValuesComplex, "[a,a,a]"), - UTF8StringModeTestCase("unicode_rtrim", bufferValuesComplex, "[a,a,a]")) - - testCasesUTF8String.foreach { t => - val buffer = new OpenHashMap[AnyRef, Long](5) - val myMode = Mode(child = Literal.create(null, StructType(Seq( - StructField("f1", StringType(t.collationId), true), - StructField("f2", StringType(t.collationId), true), - StructField("f3", StringType(t.collationId), true) - )))) - t.bufferValues.foreach { case (k, v) => buffer.update(k, v) } - assert(myMode.eval(buffer).toString.toLowerCase() == t.result.toLowerCase()) - } - } - - test("Support mode for string expression with collated strings in struct") { - case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) - val testCases = Seq( - ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("UTF8_LCASE_RTRIM", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") - ) - testCases.foreach(t => { - val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => - (0L to numRepeats).map(_ => s"named_struct('f1'," + - s" collate('$elt', '${t.collationId}'), 'f2', 1)").mkString(",") - }.mkString(",") - - val tableName = s"t_${t.collationId}_mode_struct" - withTable(tableName) { - sql(s"CREATE TABLE ${tableName}(i STRUCT) USING parquet") - sql(s"INSERT INTO ${tableName} VALUES " + valuesToAdd) - val query = s"SELECT lower(mode(i).f1) FROM ${tableName}" - checkAnswer(sql(query), Row(t.result)) - } - }) - } - - test("Support mode for string expression with collated strings in recursively nested struct") { - case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) - val testCases = Seq( - ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("UTF8_LCASE_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") - ) - testCases.foreach { t => - val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => - (0L to numRepeats).map(_ => s"named_struct('f1', " + - s"named_struct('f2', collate('$elt', '${t.collationId}')), 'f3', 1)").mkString(",") - }.mkString(",") - - val tableName = s"t_${t.collationId}_mode_nested_struct1" - withTable(tableName) { - sql(s"CREATE TABLE ${tableName}(i STRUCT, f3: INT>) USING parquet") - sql(s"INSERT INTO ${tableName} VALUES " + valuesToAdd) - val query = s"SELECT lower(mode(i).f1.f2) FROM ${tableName}" - checkAnswer(sql(query), Row(t.result)) - } - } - } - - test("Support mode for string expression with collated strings in array complex type") { - case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) - val testCases = Seq( - ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("UTF8_LCASE_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") - ) - testCases.foreach { t => - val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => - (0L to numRepeats).map(_ => s"array(named_struct('f2', " + - s"collate('$elt', '${t.collationId}'), 'f3', 1))").mkString(",") - }.mkString(",") - - val tableName = s"t_${t.collationId}_mode_nested_struct2" - withTable(tableName) { - sql(s"CREATE TABLE ${tableName}(" + - s"i ARRAY< STRUCT>)" + - s" USING parquet") - sql(s"INSERT INTO ${tableName} VALUES " + valuesToAdd) - val query = s"SELECT lower(element_at(mode(i).f2, 1)) FROM ${tableName}" - checkAnswer(sql(query), Row(t.result)) - } - } - } - - test("Support mode for string expression with collated strings in 3D array type") { - case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) - val testCases = Seq( - ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("UTF8_LCASE_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") - ) - testCases.foreach { t => - val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => - (0L to numRepeats).map(_ => - s"array(array(array(collate('$elt', '${t.collationId}'))))").mkString(",") - }.mkString(",") - - val tableName = s"t_${t.collationId}_mode_nested_3d_array" - withTable(tableName) { - sql(s"CREATE TABLE ${tableName}(i ARRAY>>) USING parquet") - sql(s"INSERT INTO ${tableName} VALUES " + valuesToAdd) - val query = s"SELECT lower(" + - s"element_at(element_at(element_at(mode(i),1),1),1)) FROM ${tableName}" - checkAnswer(sql(query), Row(t.result)) - } - } - } - - test("Support mode for string expression with collated complex type - Highly nested") { - case class ModeTestCase[R](collationId: String, bufferValues: Map[String, Long], result: R) - val testCases = Seq( - ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("UTF8_LCASE", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("UTF8_LCASE_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "a"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b"), - ModeTestCase("unicode_ci_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "b") - ) - testCases.foreach { t => - val valuesToAdd = t.bufferValues.map { case (elt, numRepeats) => - (0L to numRepeats).map(_ => s"array(named_struct('s1', named_struct('a2', " + - s"array(collate('$elt', '${t.collationId}'))), 'f3', 1))").mkString(",") - }.mkString(",") - - val tableName = s"t_${t.collationId}_mode_highly_nested_struct" - withTable(tableName) { - sql(s"CREATE TABLE ${tableName}(" + - s"i ARRAY>, f3: INT>>)" + - s" USING parquet") - sql(s"INSERT INTO ${tableName} VALUES " + valuesToAdd) - val query = s"SELECT lower(element_at(element_at(mode(i), 1).s1.a2, 1)) FROM ${tableName}" - - checkAnswer(sql(query), Row(t.result)) - } - } - } - - test("Support mode for string expression with collated complex type - nested map") { - case class ModeTestCase(collationId: String, bufferValues: Map[String, Long], result: String) - Seq( - ModeTestCase("utf8_binary", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{a -> 1}"), - ModeTestCase("utf8_binary_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{a -> 1}"), - ModeTestCase("unicode", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{a -> 1}"), - ModeTestCase("unicode_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{a -> 1}"), - ModeTestCase("utf8_lcase", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{b -> 1}"), - ModeTestCase("utf8_lcase_rtrim", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{b -> 1}"), - ModeTestCase("unicode_ci", Map("a" -> 3L, "b" -> 2L, "B" -> 2L), "{b -> 1}") - ).foreach { t1 => - def getValuesToAdd(t: ModeTestCase): String = { - val valuesToAdd = t.bufferValues.map { - case (elt, numRepeats) => - (0L to numRepeats).map(i => - s"named_struct('m1', map(collate('$elt', '${t.collationId}'), 1))" - ).mkString(",") - }.mkString(",") - valuesToAdd - } - val tableName = s"t_${t1.collationId}_mode_nested_map_struct1" - withTable(tableName) { - withSQLConf(SQLConf.ALLOW_COLLATIONS_IN_MAP_KEYS.key -> "true") { - sql(s"CREATE TABLE ${tableName}(" + - s"i STRUCT>) USING parquet") - sql(s"INSERT INTO ${tableName} VALUES ${getValuesToAdd(t1)}") - } - val query = "SELECT lower(cast(mode(i).m1 as string))" + - s" FROM ${tableName}" - val queryResult = sql(query) - checkAnswer(queryResult, Row(t1.result)) - } - } - } - - test("SPARK-48430: Map value extraction with collations") { - for { - collateKey <- Seq(true, false) - collateVal <- Seq(true, false) - defaultCollation <- Seq( - "UTF8_BINARY", - "UTF8_BINARY_RTRIM", - "UTF8_LCASE", - "UTF8_LCASE_RTRIM", - "UNICODE") - } { - val mapKey = if (collateKey) "'a' collate utf8_lcase" else "'a'" - val mapVal = if (collateVal) "'b' collate utf8_lcase" else "'b'" - val collation = if (collateVal) "UTF8_LCASE" else "UTF8_BINARY" - val queryExtractor = s"select collation(map($mapKey, $mapVal)[$mapKey])" - val queryElementAt = s"select collation(element_at(map($mapKey, $mapVal), $mapKey))" - - checkAnswer(sql(queryExtractor), Row(fullyQualifiedPrefix + collation)) - checkAnswer(sql(queryElementAt), Row(fullyQualifiedPrefix + collation)) - - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> defaultCollation) { - val res = fullyQualifiedPrefix + (if (collateVal) "UTF8_LCASE" else defaultCollation) - checkAnswer(sql(queryExtractor), Row(res)) - checkAnswer(sql(queryElementAt), Row(res)) - } - } - } - - test("CurrentTimeZone expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = "select current_timezone()" - // Data type check - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val testQuery = sql(query) - val dataType = StringType(collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("DayName expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = "select dayname(current_date())" - // Data type check - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val testQuery = sql(query) - val dataType = StringType(collationName) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - } - }) - } - - test("ToUnixTimestamp expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select to_unix_timestamp(collate('2021-01-01 00:00:00', '${collationName}'), - |collate('yyyy-MM-dd HH:mm:ss', '${collationName}')) - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = LongType - val expectedResult = 1609488000L - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(expectedResult)) - }) - } - - test("FromUnixTime expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select from_unixtime(1609488000, collate('yyyy-MM-dd HH:mm:ss', '${collationName}')) - |""".stripMargin - // Result & data type check - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val testQuery = sql(query) - val dataType = StringType(collationName) - val expectedResult = "2021-01-01 00:00:00" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(expectedResult)) - } - }) - } - - test("NextDay expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select next_day('2015-01-14', collate('TU', '${collationName}')) - |""".stripMargin - // Result & data type check - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val testQuery = sql(query) - val dataType = DateType - val expectedResult = "2015-01-20" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Date.valueOf(expectedResult))) - } - }) - } - - test("FromUTCTimestamp expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select from_utc_timestamp(collate('2016-08-31', '${collationName}'), - |collate('Asia/Seoul', '${collationName}')) - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = TimestampType - val expectedResult = "2016-08-31 09:00:00.0" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Timestamp.valueOf(expectedResult))) - }) - } - - test("ToUTCTimestamp expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select to_utc_timestamp(collate('2016-08-31 09:00:00', '${collationName}'), - |collate('Asia/Seoul', '${collationName}')) - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = TimestampType - val expectedResult = "2016-08-31 00:00:00.0" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Timestamp.valueOf(expectedResult))) - }) - } - - test("ParseToDate expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select to_date(collate('2016-12-31', '${collationName}'), - |collate('yyyy-MM-dd', '${collationName}')) - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = DateType - val expectedResult = "2016-12-31" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Date.valueOf(expectedResult))) - }) - } - - test("ParseToTimestamp expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select to_timestamp(collate('2016-12-31 23:59:59', '${collationName}'), - |collate('yyyy-MM-dd HH:mm:ss', '${collationName}')) - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = TimestampType - val expectedResult = "2016-12-31 23:59:59.0" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Timestamp.valueOf(expectedResult))) - }) - } - - test("TruncDate expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select trunc(collate('2016-12-31 23:59:59', '${collationName}'), 'MM') - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = DateType - val expectedResult = "2016-12-01" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Date.valueOf(expectedResult))) - }) - } - - test("TruncTimestamp expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select date_trunc(collate('HOUR', '${collationName}'), - |collate('2015-03-05T09:32:05.359', '${collationName}')) - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = TimestampType - val expectedResult = "2015-03-05 09:00:00.0" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Timestamp.valueOf(expectedResult))) - }) - } - - test("MakeTimestamp expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select make_timestamp(2014, 12, 28, 6, 30, 45.887, collate('CET', '${collationName}')) - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = TimestampType - val expectedResult = "2014-12-27 21:30:45.887" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Timestamp.valueOf(expectedResult))) - }) - } - - test("ExtractValue expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val query = - s""" - |select col['Field1'] - |from values (named_struct('Field1', 'Spark', 'Field2', 5)) as tab(col); - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = StringType(collationName) - val expectedResult = "Spark" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(expectedResult)) - } - }) - } - - test("Lag expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |SELECT lag(a, -1, 'default' collate $collationName) OVER (PARTITION BY b ORDER BY a) - |FROM VALUES ('A1', 2), ('A2', 1), ('A2', 3), ('A1', 1) tab(a, b); - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = StringType(collationName) - val expectedResult = Seq("A2", "default", "default", "default") - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, expectedResult.map(Row(_))) - }) - } - - test("Lead expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |SELECT lead(a, -1, 'default' collate $collationName) OVER (PARTITION BY b ORDER BY a) - |FROM VALUES ('A1', 2), ('A2', 1), ('A2', 3), ('A1', 1) tab(a, b); - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = StringType(collationName) - val expectedResult = Seq("A1", "default", "default", "default") - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, expectedResult.map(Row(_))) - }) - } - - test("DatePart expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select date_part(collate('Week', '${collationName}'), - |collate('2019-08-12 01:00:00.123456', '${collationName}')) - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = IntegerType - val expectedResult = 33 - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(expectedResult)) - }) - } - - test("DateAdd expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = s"""select date_add(collate('2016-07-30', '${collationName}'), 1)""" - // Result & data type check - val testQuery = sql(query) - val dataType = DateType - val expectedResult = "2016-07-31" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Date.valueOf(expectedResult))) - }) - } - - test("DateSub expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = s"""select date_sub(collate('2016-07-30', '${collationName}'), 1)""" - // Result & data type check - val testQuery = sql(query) - val dataType = DateType - val expectedResult = "2016-07-29" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(Date.valueOf(expectedResult))) - }) - } - - test("WindowTime and TimeWindow expressions with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val query = - s"""SELECT window_time(window) - | FROM (SELECT a, window, count(*) as cnt FROM VALUES - |('A1', '2021-01-01 00:00:00'), - |('A1', '2021-01-01 00:04:30'), - |('A1', '2021-01-01 00:06:00'), - |('A2', '2021-01-01 00:01:00') AS tab(a, b) - |GROUP by a, window(b, '5 minutes') ORDER BY a, window.start); - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = TimestampType - val expectedResults = - Seq("2021-01-01 00:04:59.999999", - "2021-01-01 00:09:59.999999", - "2021-01-01 00:04:59.999999") - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, expectedResults.map(ts => Row(Timestamp.valueOf(ts)))) - } - }) - } - - test("SessionWindow expressions with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { - val query = - s"""SELECT count(*) as cnt - | FROM VALUES - |('A1', '2021-01-01 00:00:00'), - |('A1', '2021-01-01 00:04:30'), - |('A1', '2021-01-01 00:10:00'), - |('A2', '2021-01-01 00:01:00'), - |('A2', '2021-01-01 00:04:30') AS tab(a, b) - |GROUP BY a, - |session_window(b, CASE WHEN a = 'A1' THEN '5 minutes' ELSE '1 minutes' END) - |ORDER BY a, session_window.start; - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = LongType - val expectedResults = Seq(2, 1, 1, 1) - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, expectedResults.map(Row(_))) - } - }) - } - - test("ConvertTimezone expression with collation") { - // Supported collations - testSuppCollations.foreach(collationName => { - val query = - s""" - |select date_format(convert_timezone(collate('America/Los_Angeles', '${collationName}'), - |collate('UTC', '${collationName}'), collate('2021-12-06 00:00:00', '${collationName}')), - |'yyyy-MM-dd HH:mm:ss.S') - |""".stripMargin - // Result & data type check - val testQuery = sql(query) - val dataType = StringType - val expectedResult = "2021-12-06 08:00:00.0" - assert(testQuery.schema.fields.head.dataType.sameType(dataType)) - checkAnswer(testQuery, Row(expectedResult)) - }) - } - - test("Reflect expressions with collated strings") { - // be aware that output of java.util.UUID.fromString is always lowercase - - case class ReflectExpressions( - left: String, - leftCollation: String, - right: String, - rightCollation: String, - result: Boolean - ) - - val testCases = Seq( - ReflectExpressions("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", - "a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", true), - ReflectExpressions("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", - "A5Cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", false), - ReflectExpressions("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary", - "a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_binary_rtrim", true), - ReflectExpressions("A5cf6C42-0C85-418f-af6c-3E4E5b1328f2", "utf8_binary", - "a5cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_lcase", true), - ReflectExpressions("A5cf6C42-0C85-418f-af6c-3E4E5b1328f2", "utf8_binary", - "A5Cf6c42-0c85-418f-af6c-3e4e5b1328f2", "utf8_lcase", true) - ) - testCases.foreach(testCase => { - val query = - s""" - |SELECT REFLECT('java.util.UUID', 'fromString', - |collate('${testCase.left}', '${testCase.leftCollation}'))= - |collate('${testCase.right}', '${testCase.rightCollation}'); - |""".stripMargin - - if (testCase.leftCollation == testCase.rightCollation) { - checkAnswer(sql(query), Row(testCase.result)) - } else { - val exception = intercept[AnalysisException] { - sql(query) - } - assert(exception.getCondition === "COLLATION_MISMATCH.EXPLICIT") - } - }) - - val queryPass = - s""" - |SELECT REFLECT('java.lang.Integer', 'toHexString',2); - |""".stripMargin - val testQueryPass = sql(queryPass) - checkAnswer(testQueryPass, Row("2")) - - val queryFail = - s""" - |SELECT REFLECT('java.lang.Integer', 'toHexString',"2"); - |""".stripMargin - checkError( - exception = intercept[ExtendedAnalysisException] { - sql(queryFail).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_STATIC_METHOD", - parameters = Map( - "methodName" -> "toHexString", - "className" -> "java.lang.Integer", - "sqlExpr" -> "\"reflect(java.lang.Integer, toHexString, 2)\""), - context = ExpectedContext( - fragment = """REFLECT('java.lang.Integer', 'toHexString',"2")""", - start = 8, - stop = 54) - ) - } - - // common method for subsequent tests verifying various SQL expressions with collations - private def testCollationSqlExpressionCommon( - query: String, - collation: String, - result: Row, - expectedType: DataType): Unit = { - testCollationSqlExpressionCommon(query, collation, Seq(result), Seq(expectedType)) - } - - // common method for subsequent tests verifying various SQL expressions with collations - private def testCollationSqlExpressionCommon( - query: String, - collation: String, - result: Seq[Row], - expectedTypes: Seq[DataType]): Unit = { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collation) { - // check result correctness - checkAnswer(sql(query), result) - // check result rows data types - for (i <- 0 until expectedTypes.length) - assert(sql(query).schema(i).dataType == expectedTypes(i)) - } - } - - test("min_by supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT min_by(x, y) FROM VALUES ('a', 10), ('b', 50), ('c', 20) AS tab(x, y)", - collation, - result = Row("a"), - expectedType = StringType(collation) - ) - } - } - - test("max_by supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT max_by(x, y) FROM VALUES ('a', 10), ('b', 50), ('c', 20) AS tab(x, y)", - collation, - result = Row("b"), - expectedType = StringType(collation) - ) - } - } - - test("array supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array('a', 'b', 'c')", - collation, - result = Row(Seq("a", "b", "c")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("array_agg supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_agg(col) FROM VALUES ('a'), ('b'), ('c') AS tab(col)", - collation, - result = Row(Seq("a", "b", "c")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("array_contains supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_contains(array('a', 'b', 'c'), 'b')", - collation, - result = Row(true), - expectedType = BooleanType - ) - } - } - - test("arrays_overlap supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT arrays_overlap(array('a', 'b', 'c'), array('c', 'd', 'e'))", - collation, - result = Row(true), - expectedType = BooleanType - ) - } - } - - test("array_insert supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_insert(array('a', 'b', 'c', 'd'), 5, 'e')", - collation, - result = Row(Seq("a", "b", "c", "d", "e")), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("array_intersect supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_intersect(array('a', 'b', 'c'), array('b', 'c', 'd'))", - collation, - result = Row(Seq("b", "c")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("array_join supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_join(array('hello', 'world'), ' ')", - collation, - result = Row("hello world"), - expectedType = StringType(collation) - ) - } - } - - test("array_position supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_position(array('a', 'b', 'c', 'c'), 'c')", - collation, - result = Row(3), - expectedType = LongType - ) - } - } - - test("array_size supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_size(array('a', 'b', 'c', 'c'))", - collation, - result = Row(4), - expectedType = IntegerType - ) - } - } - - test("array_sort supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_sort(array('b', null, 'A'))", - collation, - result = Row(Seq("A", "b", null)), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("array_except supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_except(array('a', 'b', 'c'), array('c', 'd', 'e'))", - collation, - result = Row(Seq("a", "b")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("array_union supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_union(array('a', 'b', 'c'), array('a', 'c', 'd'))", - collation, - result = Row(Seq("a", "b", "c", "d")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("array_compact supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_compact(array('a', 'b', null, 'c'))", - collation, - result = Row(Seq("a", "b", "c")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("arrays_zip supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT arrays_zip(array('a', 'b', 'c'), array(1, 2, 3))", - collation, - result = Row(Seq(Row("a", 1), Row("b", 2), Row("c", 3))), - expectedType = ArrayType(StructType( - StructField("0", StringType(collation), true) :: - StructField("1", IntegerType, true) :: Nil - ), false) - ) - } - } - - test("array_min supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_min(array('a', 'b', null, 'c'))", - collation, - result = Row("a"), - expectedType = StringType(collation) - ) - } - } - - test("array_max supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_max(array('a', 'b', null, 'c'))", - collation, - result = Row("c"), - expectedType = StringType(collation) - ) - } - } - - test("array_append supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_append(array('b', 'd', 'c', 'a'), 'e')", - collation, - result = Row(Seq("b", "d", "c", "a", "e")), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("array_repeat supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_repeat('abc', 2)", - collation, - result = Row(Seq("abc", "abc")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("array_remove supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_remove(array('a', 'b', null, 'c'), 'b')", - collation, - result = Row(Seq("a", null, "c")), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("array_prepend supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_prepend(array('b', 'd', 'c', 'a'), 'd')", - collation, - result = Row(Seq("d", "b", "d", "c", "a")), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("array_distinct supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT array_distinct(array('a', 'b', 'c', null, 'c'))", - collation, - result = Row(Seq("a", "b", "c", null)), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("collect_list supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT collect_list(col) FROM VALUES ('a'), ('b'), ('c') AS tab(col)", - collation, - result = Row(Seq("a", "b", "c")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("collect_set does not support collation") { - testAdditionalCollations.foreach { collation => - val query = "SELECT collect_set(col) FROM VALUES ('a'), ('b'), ('a') AS tab(col);" - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collation) { - checkError( - exception = intercept[AnalysisException] { - sql(query) - }, - condition = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "functionName" -> "`collect_set`", - "dataType" -> "\"MAP\" or \"COLLATED STRING\"", - "sqlExpr" -> "\"collect_set(col)\""), - context = ExpectedContext( - fragment = "collect_set(col)", - start = 7, - stop = 22)) - } - } - } - - test("element_at supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT element_at(array('a', 'b', 'c'), 2)", - collation, - result = Row("b"), - expectedType = StringType(collation) - ) - } - } - - test("aggregate supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT aggregate(array('a', 'b', 'c'), '', (acc, x) -> concat(acc, x))", - collation, - result = Row("abc"), - expectedType = StringType(collation) - ) - } - } - - test("explode supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT explode(array('a', 'b'))", - collation, - result = Seq( - Row("a"), - Row("b") - ), - expectedTypes = Seq( - StringType(collation) - ) - ) - } - } - - test("posexplode supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT posexplode(array('a', 'b'))", - collation, - result = Seq( - Row(0, "a"), - Row(1, "b") - ), - expectedTypes = Seq( - IntegerType, - StringType(collation) - ) - ) - } - } - - test("filter supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT filter(array('a', 'b', 'c'), x -> x < 'b')", - collation, - result = Row(Seq("a")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("flatten supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT flatten(array(array('a', 'b'), array('c', 'd')))", - collation, - result = Row(Seq("a", "b", "c", "d")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("inline supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT inline(array(struct(1, 'a'), struct(2, 'b')))", - collation, - Seq( - Row(1, "a"), - Row(2, "b") - ), - expectedTypes = Seq( - IntegerType, - StringType(collation) - ) - ) - } - } - - test("shuffle supports collation") { - testAdditionalCollations.foreach { collation => - val query = "SELECT shuffle(array('a', 'b', 'c', 'd'));" - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collation) { - // check result row data type - val dataType = ArrayType(StringType(collation), false) - assert(sql(query).schema.head.dataType == dataType) - } - } - } - - test("slice supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT slice(array('a', 'b', 'c', 'd'), 2, 2)", - collation, - result = Row(Seq("b", "c")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("sort_array supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT sort_array(array('b', 'd', null, 'c', 'a'), true)", - collation, - result = Row(Seq(null, "a", "b", "c", "d")), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("zip_with supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT zip_with(array('a', 'b'), array('x', 'y'), (x, y) -> concat(x, y))", - collation, - result = Row(Seq("ax", "by")), - expectedType = ArrayType( - StringType(collation), - containsNull = true - ) - ) - } - } - - test("map_contains_key supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_contains_key(map('a', 1, 'b', 2), 'a')", - collation, - result = Row(true), - expectedType = BooleanType - ) - } - } - - test("map_from_arrays supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_from_arrays(array('a','b','c'), array(1,2,3))", - collation, - result = Row(Map("a" -> 1, "b" -> 2, "c" -> 3)), - expectedType = MapType( - StringType(collation), - IntegerType, false - ) - ) - } - } - - test("map_keys supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_keys(map('a', 1, 'b', 2))", - collation, - result = Row(Seq("a", "b")), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("map_values supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_values(map(1, 'a', 2, 'b'))", - collation, - result = Row(Seq("a", "b")), - expectedType = ArrayType(StringType(collation), true) - ) - } - } - - test("map_entries supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_entries(map('a', 1, 'b', 2))", - collation, - result = Row(Seq(Row("a", 1), Row("b", 2))), - expectedType = ArrayType(StructType( - StructField("key", StringType(collation), false) :: - StructField("value", IntegerType, false) :: Nil - ), false) - ) - } - } - - test("map_from_entries supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_from_entries(array(struct(1, 'a'), struct(2, 'b')))", - collation, - result = Row(Map(1 -> "a", 2 -> "b")), - expectedType = MapType( - IntegerType, - StringType(collation), - valueContainsNull = false - ) - ) - } - } - - test("map_concat supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_concat(map(1, 'a'), map(2, 'b'))", - collation, - result = Row(Map(1 -> "a", 2 -> "b")), - expectedType = MapType( - IntegerType, - StringType(collation), - valueContainsNull = false - ) - ) - } - } - - test("map_filter supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_filter(map('a', 1, 'b', 2, 'c', 3), (k, v) -> k < 'c')", - collation, - result = Row(Map("a" -> 1, "b" -> 2)), - expectedType = MapType( - StringType(collation), - IntegerType, - valueContainsNull = false - ) - ) - } - } - - test("map_zip_with supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT map_zip_with(map(1, 'a'), map(1, 'x'), (k, v1, v2) -> concat(v1, v2))", - collation, - result = Row(Map(1 -> "ax")), - expectedType = MapType( - IntegerType, - StringType(collation), - valueContainsNull = true - ) - ) - } - } - - test("transform supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT transform(array('aa', 'bb', 'cc'), x -> substring(x, 2))", - collation, - result = Row(Seq("a", "b", "c")), - expectedType = ArrayType(StringType(collation), false) - ) - } - } - - test("transform_values supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT transform_values(map_from_arrays(array(1, 2, 3)," + - "array('aa', 'bb', 'cc')), (k, v) -> substring(v, 2))", - collation, - result = Row(Map(1 -> "a", 2 -> "b", 3 -> "c")), - expectedType = MapType( - IntegerType, - StringType(collation), - false - ) - ) - } - } - - test("transform_keys supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT transform_keys(map_from_arrays(array('aa', 'bb', 'cc')," + - "array(1, 2, 3)), (k, v) -> substring(k, 2))", - collation, - result = Row(Map("a" -> 1, "b" -> 2, "c" -> 3)), - expectedType = MapType( - StringType(collation), - IntegerType, - false - ) - ) - } - } - - test("stack supports collation") { - testAdditionalCollations.foreach { collation => - testCollationSqlExpressionCommon( - query = "SELECT stack(2, 'a', 'b', 'c')", - collation, - result = Seq( - Row("a", "b"), - Row("c", null) - ), - expectedTypes = Seq( - StringType(collation) - ) - ) - } - } - - test("SPARK-50060: set operators with conflicting collations") { - val setOperators = Seq[(String, Int, Int)]( - ("UNION", 64, 45), - ("INTERSECT", 68, 49), - ("EXCEPT", 65, 46)) - - for { - ansiEnabled <- Seq(true, false) - (operator, stopExplicit, stopDefault) <- setOperators - } { - withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString, - SqlApiConf.DEFAULT_COLLATION -> "UNICODE_CI") { - val explicitConflictQuery = - s"SELECT 'a' COLLATE UTF8_LCASE $operator SELECT 'A' COLLATE UNICODE_CI" - checkError( - exception = intercept[AnalysisException] { - sql(explicitConflictQuery) - }, - condition = "INCOMPATIBLE_COLUMN_TYPE", - parameters = Map( - "columnOrdinalNumber" -> "first", - "tableOrdinalNumber" -> "second", - "dataType1" -> "\"STRING COLLATE UNICODE_CI\"", - "dataType2" -> "\"STRING COLLATE UTF8_LCASE\"", - "operator" -> operator, - "hint" -> ""), - context = ExpectedContext( - fragment = explicitConflictQuery, - start = 0, - stop = stopExplicit)) - - val defaultConflictQuery = - s"SELECT 'a' COLLATE UTF8_LCASE $operator SELECT 'A'" - checkError( - exception = intercept[AnalysisException] { - sql(defaultConflictQuery) - }, - condition = "INCOMPATIBLE_COLUMN_TYPE", - parameters = Map( - "columnOrdinalNumber" -> "first", - "tableOrdinalNumber" -> "second", - "dataType1" -> "\"STRING COLLATE UNICODE_CI\"", - "dataType2" -> "\"STRING COLLATE UTF8_LCASE\"", - "operator" -> operator, - "hint" -> ""), - context = ExpectedContext( - fragment = defaultConflictQuery, - start = 0, - stop = stopDefault)) - } - } - } - - test("Support HyperLogLogPlusPlus expression with collation") { - case class HyperLogLogPlusPlusTestCase( - collation: String, - input: Seq[String], - output: Seq[Row] - ) - - val testCases = Seq( - HyperLogLogPlusPlusTestCase("utf8_binary", Seq("a", "a", "A", "z", "zz", "ZZ", "w", - "AA", "aA", "Aa", "aa"), Seq(Row(10))), - HyperLogLogPlusPlusTestCase("utf8_binary_rtrim", Seq("a ", "a", "a", "A", "z", "zz", "ZZ", - "w", "AA", "aA", "Aa", "aa"), Seq(Row(10))), - HyperLogLogPlusPlusTestCase("utf8_lcase", Seq("a", "a", "A", "z", "zz", "ZZ", "w", - "AA", "aA", "Aa", "aa"), Seq(Row(5))), - HyperLogLogPlusPlusTestCase("utf8_lcase_rtrim", Seq("a ", "a", "a", "A", "z", "zz", "ZZ", "w", - "AA", "aA", "Aa", "aa"), Seq(Row(5))), - HyperLogLogPlusPlusTestCase("UNICODE", Seq("a", "a", "A", "z", "zz", "ZZ", "w", "AA", - "aA", "Aa", "aa"), Seq(Row(9))), - HyperLogLogPlusPlusTestCase("UNICODE_RTRIM", Seq("a ", "a", "a", "A", "z", "zz", "ZZ", "w", - "AA", "aA", "Aa", "aa"), Seq(Row(9))), - HyperLogLogPlusPlusTestCase("UNICODE_CI", Seq("a", "a", "A", "z", "zz", "ZZ", "w", "AA", - "aA", "Aa", "aa"), Seq(Row(5))), - HyperLogLogPlusPlusTestCase("UNICODE_CI_RTRIM", Seq("a ", "a", "a", "A", "z", "zz", "ZZ", "w", - "AA", "aA", "Aa", "aa"), Seq(Row(5))) - ) - - testCases.foreach( t => { - // Using explicit collate clause - val query = - s""" - |SELECT approx_count_distinct(col) FROM VALUES - |${t.input.map(s => s"('${s}' collate ${t.collation})").mkString(", ") } tab(col) - |""".stripMargin - checkAnswer(sql(query), t.output) - - // Using default collation - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collation) { - val query = - s""" - |SELECT approx_count_distinct(col) FROM VALUES - |${t.input.map(s => s"('${s}')").mkString(", ") } tab(col) - |""".stripMargin - checkAnswer(sql(query), t.output) - } - }) - } - - // TODO: Add more tests for other SQL expressions - -} -// scalastyle:on nonascii - -class CollationSQLExpressionsANSIOffSuite extends CollationSQLExpressionsSuite { - override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.ANSI_ENABLED, false) - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala index 6ca86b91edb17..87c399f895ada 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.Project -import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{ArrayType, BooleanType, IntegerType, StringType} @@ -99,15 +98,13 @@ class CollationSQLRegexpSuite test("Like simplification should work with collated strings (for default collation)") { val tableNameBinary = "T_BINARY" withTable(tableNameBinary) { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UTF8_BINARY") { - sql(s"CREATE TABLE IF NOT EXISTS $tableNameBinary(c STRING) using PARQUET") - sql(s"INSERT INTO $tableNameBinary(c) VALUES('ABC')") - checkAnswer(sql(s"select c like 'ab%' FROM $tableNameBinary"), Row(false)) - checkAnswer(sql(s"select c like '%bc' FROM $tableNameBinary"), Row(false)) - checkAnswer(sql(s"select c like 'a%c' FROM $tableNameBinary"), Row(false)) - checkAnswer(sql(s"select c like '%b%' FROM $tableNameBinary"), Row(false)) - checkAnswer(sql(s"select c like 'abc' FROM $tableNameBinary"), Row(false)) - } + sql(s"CREATE TABLE IF NOT EXISTS $tableNameBinary(c STRING) using PARQUET") + sql(s"INSERT INTO $tableNameBinary(c) VALUES('ABC')") + checkAnswer(sql(s"select c like 'ab%' FROM $tableNameBinary"), Row(false)) + checkAnswer(sql(s"select c like '%bc' FROM $tableNameBinary"), Row(false)) + checkAnswer(sql(s"select c like 'a%c' FROM $tableNameBinary"), Row(false)) + checkAnswer(sql(s"select c like '%b%' FROM $tableNameBinary"), Row(false)) + checkAnswer(sql(s"select c like 'abc' FROM $tableNameBinary"), Row(false)) } val tableNameLcase = "T_LCASE" withTable(tableNameLcase) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala deleted file mode 100644 index 6ef15e5b6d09a..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala +++ /dev/null @@ -1,1782 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import org.apache.spark.SparkConf -import org.apache.spark.sql.catalyst.analysis.CollationTypeCasts -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types._ - -// scalastyle:off nonascii -class CollationStringExpressionsSuite - extends QueryTest - with SharedSparkSession - with ExpressionEvalHelper { - - test("Support `ConcatWs` string expression with collation") { - case class ConcatWsTestCase[R]( - sep: String, - arrayStr: Array[String], - collation: String, - result: R) - val testCases = Seq( - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UTF8_BINARY", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UTF8_BINARY_RTRIM", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UTF8_LCASE", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UTF8_LCASE_RTRIM", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UNICODE", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UNICODE_RTRIM", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UNICODE_CI", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UNICODE_CI_RTRIM", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UNICODE_CI", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "Unterstützung"), "DE_CI_AI", "Spark Unterstützung") - ) - testCases.foreach(t => { - // Unit test. - val inputExprs = t.arrayStr.map { - case null => Literal.create(null, StringType(t.collation)) - case s: String => Literal.create(s, StringType(t.collation)) - } - val sepExpr = Literal.create(t.sep, StringType(t.collation)) - checkEvaluation(ConcatWs(sepExpr +: inputExprs.toIndexedSeq), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val array = t.arrayStr.map(s => s"'$s'").mkString(", ") - val query = s"select concat_ws('${t.sep}', $array)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Elt` string expression with collation") { - case class EltTestCase[R](index: Integer, inputs: Array[String], collation: String, result: R) - val testCases = Seq( - EltTestCase(1, Array("Spark", "SQL"), "UTF8_BINARY", "Spark"), - EltTestCase(1, Array("Spark", "SQL"), "UTF8_BINARY_RTRIM", "Spark"), - EltTestCase(1, Array("Spark", "SQL"), "UTF8_LCASE", "Spark"), - EltTestCase(1, Array("Spark", "SQL"), "UTF8_LCASE_RTRIM", "Spark"), - EltTestCase(2, Array("Spark", "SQL"), "UNICODE", "SQL"), - EltTestCase(2, Array("Spark", "SQL"), "UNICODE_RTRIM", "SQL"), - EltTestCase(2, Array("Spark", "SQL"), "UNICODE_CI", "SQL"), - EltTestCase(2, Array("Spark", "SQL"), "UNICODE_CI_RTRIM", "SQL"), - EltTestCase(2, Array("Spark", "SQL"), "UNICODE_CI", "SQL"), - EltTestCase(2, Array("Spark", "Unterstützung"), "DE_CI", "Unterstützung") - ) - testCases.foreach(t => { - // Unit test. - val inputExprs = t.inputs.map { - case null => Literal.create(null, StringType(t.collation)) - case s: String => Literal.create(s, StringType(t.collation)) - } - val intExpr = Literal.create(t.index, IntegerType) - checkEvaluation(Elt(intExpr +: inputExprs.toIndexedSeq), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select elt(${t.index}, '${t.inputs(0)}', '${t.inputs(1)}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `SplitPart` string expression with collation") { - case class SplitPartTestCase[R]( - str: String, - delimiter: String, - partNum: Integer, - collation: String, - result: R) - val testCases = Seq( - SplitPartTestCase("1a2", "a", 2, "UTF8_BINARY", "2"), - SplitPartTestCase("1a2", "a ", 1, "UTF8_BINARY_RTRIM", "1"), - SplitPartTestCase("1a2", "a", 2, "UNICODE", "2"), - SplitPartTestCase("1a 2", "a ", 2, "UNICODE_RTRIM", " 2"), - SplitPartTestCase("1a2", "A", 2, "UTF8_LCASE", "2"), - SplitPartTestCase("1 a2", "A ", 2, "UTF8_LCASE_RTRIM", "2"), - SplitPartTestCase("1a2", "A", 2, "UNICODE_CI", "2"), - SplitPartTestCase("1 a2 ", "A ", 2, "UNICODE_CI_RTRIM", "2 "), - SplitPartTestCase("1a2", "A", 2, "UNICODE_CI", "2"), - SplitPartTestCase("1ö2", "O", 2, "DE_CI_AI", "2") - ) - val unsupportedTestCase = SplitPartTestCase("1a2", "a", 2, "UNICODE_AI", "2") - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType(t.collation)) - val delimiter = Literal.create(t.delimiter, StringType(t.collation)) - val partNum = Literal.create(t.partNum, IntegerType) - checkEvaluation(SplitPart(str, delimiter, partNum), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select split_part('${t.str}', '${t.delimiter}', ${t.partNum})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select split_part('${unsupportedTestCase.str}', '${unsupportedTestCase.delimiter}', " + - s"${unsupportedTestCase.partNum})" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> "\"split_part('1a2' collate UNICODE_AI, 'a' collate UNICODE_AI, 2)\"", - "paramIndex" -> "first", - "inputSql" -> "\"'1a2' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "split_part('1a2', 'a', 2)", start = 7, stop = 31) - ) - } - } - - test("Support `StringSplitSQL` string expression with collation") { - case class StringSplitSQLTestCase[R]( - str: String, - delimiter: String, - collation: String, - result: R) - val testCases = Seq( - StringSplitSQLTestCase("1a2", "a", "UTF8_BINARY", Array("1", "2")), - StringSplitSQLTestCase("1a2", "a ", "UTF8_BINARY_RTRIM", Array("1", "2")), - StringSplitSQLTestCase("1a2", "a", "UNICODE", Array("1", "2")), - StringSplitSQLTestCase("1a 2", "a ", "UNICODE_RTRIM", Array("1", " 2")), - StringSplitSQLTestCase("1a2", "A", "UTF8_LCASE", Array("1", "2")), - StringSplitSQLTestCase("1 a2", "A ", "UTF8_LCASE_RTRIM", Array("1 ", "2")), - StringSplitSQLTestCase("1a2", "A", "UNICODE_CI", Array("1", "2")), - StringSplitSQLTestCase("1 a2 ", "A ", "UNICODE_CI_RTRIM", Array("1 ", "2 ")), - StringSplitSQLTestCase("1a2", "A", "UNICODE_CI", Array("1", "2")), - StringSplitSQLTestCase("1ä2", "Ä", "DE_CI", Array("1", "2")), - StringSplitSQLTestCase("1ä2", "A", "DE_CI_AI", Array("1", "2")) - ) - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType(t.collation)) - val delimiter = Literal.create(t.delimiter, StringType(t.collation)) - checkEvaluation(StringSplitSQL(str, delimiter), t.result) - }) - - checkError( - exception = intercept[AnalysisException] { - val expr = StringSplitSQL( - Collate(Literal.create("1a2", StringType("UTF8_BINARY")), - ResolvedCollation("UTF8_BINARY")), - Collate(Literal.create("a", StringType("UTF8_BINARY")), - ResolvedCollation("UTF8_LCASE"))) - CollationTypeCasts.transform(expr) - }, - condition = "COLLATION_MISMATCH.EXPLICIT", - sqlState = "42P21", - parameters = Map( - "explicitTypes" -> """"STRING", "STRING COLLATE UTF8_LCASE"""" - ) - ) - } - - test("Support `Contains` string expression with collation") { - case class ContainsTestCase[R](left: String, right: String, collation: String, result: R) - val testCases = Seq( - ContainsTestCase("", "", "UTF8_BINARY", true), - ContainsTestCase("", " ", "UTF8_BINARY_RTRIM", true), - ContainsTestCase("abcde", "C", "UNICODE", false), - ContainsTestCase("abcde", " C ", "UNICODE_RTRIM", false), - ContainsTestCase("abcde", "FGH", "UTF8_LCASE", false), - ContainsTestCase("abcde", "ABC ", "UTF8_LCASE_RTRIM", true), - ContainsTestCase("abcde", "BCD", "UNICODE_CI", true), - ContainsTestCase("ab c de ", "B C D ", "UNICODE_CI_RTRIM", true), - ContainsTestCase("abcde", "BCD", "UNICODE_CI", true), - ContainsTestCase("Priča o Maču u kamenu", "MAC", "SR_CI_AI", true), - ContainsTestCase("Priča o Maču u kamenu", "MAC", "SR_CI", false), - ContainsTestCase("Priča o Maču u kamenu", "MAČ", "SR", false), - ContainsTestCase("Priča o Maču u kamenu", "Mač", "SR", true), - ContainsTestCase("Прича о Мачу у камену", "мач", "sr_Cyrl_CI_AI", true), - ContainsTestCase("Прича о Мачу у камену", "мац", "sr_Cyrl_CI_AI", false) - ) - val unsupportedTestCase = ContainsTestCase("abcde", "A", "UNICODE_AI", false) - testCases.foreach(t => { - // Unit test. - val left = Literal.create(t.left, StringType(t.collation)) - val right = Literal.create(t.right, StringType(t.collation)) - checkEvaluation(Contains(left, right), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select contains('${t.left}', '${t.right}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(BooleanType)) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select contains('${unsupportedTestCase.left}', '${unsupportedTestCase.right}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> "\"contains('abcde' collate UNICODE_AI, 'A' collate UNICODE_AI)\"", - "paramIndex" -> "first", - "inputSql" -> "\"'abcde' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "contains('abcde', 'A')", start = 7, stop = 28) - ) - } - } - - test("Support `SubstringIndex` expression with collation") { - case class SubstringIndexTestCase[R]( - strExpr: String, - delimExpr: String, - countExpr: Integer, - collation: String, - result: R) - val testCases = Seq( - SubstringIndexTestCase("wwwgapachegorg", "g", -3, "UTF8_BINARY", "apachegorg"), - SubstringIndexTestCase("www||apache||org", "||", 2, "UTF8_BINARY", "www||apache"), - SubstringIndexTestCase("wwwgapachegorg", "g ", -3, "UTF8_BINARY_RTRIM", "apachegorg"), - SubstringIndexTestCase("www ||apache||org", "|| ", 2, "UTF8_BINARY_RTRIM", "www ||apache"), - SubstringIndexTestCase("wwwXapacheXorg", "x", 2, "UTF8_LCASE", "wwwXapache"), - SubstringIndexTestCase("AAA ", "a ", -2, "UTF8_LCASE_RTRIM", "A "), - SubstringIndexTestCase("aaaaaaaaaa", "aa", 2, "UNICODE", "a"), - SubstringIndexTestCase("aaaaaaaaaa ", "aa ", 2, "UNICODE_RTRIM", "a"), - SubstringIndexTestCase("wwwmapacheMorg", "M", -2, "UNICODE_CI", "apacheMorg"), - SubstringIndexTestCase("AA A ", "a ", -2, "UNICODE_CI_RTRIM", " A "), - SubstringIndexTestCase("wwwmapacheMorg", "M", -2, "UNICODE_CI", "apacheMorg"), - SubstringIndexTestCase("wwwüapacheüorg", "U", 2, "DE_CI_AI", "wwwüapache") - ) - val unsupportedTestCase = SubstringIndexTestCase("abacde", "a", 2, "UNICODE_AI", "cde") - testCases.foreach(t => { - // Unit test. - val strExpr = Literal.create(t.strExpr, StringType(t.collation)) - val delimExpr = Literal.create(t.delimExpr, StringType(t.collation)) - val countExpr = Literal.create(t.countExpr, IntegerType) - checkEvaluation(SubstringIndex(strExpr, delimExpr, countExpr), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select substring_index('${t.strExpr}', '${t.delimExpr}', ${t.countExpr})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select substring_index('${unsupportedTestCase.strExpr}', " + - s"'${unsupportedTestCase.delimExpr}', ${unsupportedTestCase.countExpr})" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> ("\"substring_index('abacde' collate UNICODE_AI, " + - "'a' collate UNICODE_AI, 2)\""), - "paramIndex" -> "first", - "inputSql" -> "\"'abacde' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext( - fragment = "substring_index('abacde', 'a', 2)", - start = 7, - stop = 39)) - } - } - - test("Support `StringInStr` string expression with collation") { - case class StringInStrTestCase[R](str: String, substr: String, collation: String, result: R) - val testCases = Seq( - StringInStrTestCase("test大千世界X大千世界", "大千", "UTF8_BINARY", 5), - StringInStrTestCase("test大千世界X大千世界", "大千 ", "UTF8_BINARY_RTRIM", 5), - StringInStrTestCase("test大千世界X大千世界", "界x", "UTF8_LCASE", 8), - StringInStrTestCase(" test大千世界X大千世界 ", "界x ", "UTF8_LCASE_RTRIM", 9), - StringInStrTestCase("test大千世界X大千世界", "界x", "UNICODE", 0), - StringInStrTestCase("test大千世界X大千世界", "界x", "UNICODE_RTRIM", 0), - StringInStrTestCase("test大千世界X大千世界", "界y", "UNICODE_CI", 0), - StringInStrTestCase("test大千世界X大千世界", "界x", "UNICODE_CI", 8), - StringInStrTestCase("abİo12", "i̇o", "UNICODE_CI", 3), - StringInStrTestCase("test大千世界X大千世界", "大 ", "UNICODE_CI_RTRIM", 5), - StringInStrTestCase("test大千世界X大千世界", " 大 ", "UNICODE_CI_RTRIM", 0) - ) - val unsupportedTestCase = StringInStrTestCase("a", "abcde", "UNICODE_AI", 0) - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType(t.collation)) - val substr = Literal.create(t.substr, StringType(t.collation)) - checkEvaluation(StringInstr(str, substr), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select instr('${t.str}', '${t.substr}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(IntegerType)) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select instr('${unsupportedTestCase.str}', '${unsupportedTestCase.substr}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> "\"instr('a' collate UNICODE_AI, 'abcde' collate UNICODE_AI)\"", - "paramIndex" -> "first", - "inputSql" -> "\"'a' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "instr('a', 'abcde')", start = 7, stop = 25) - ) - } - } - - test("Support `FindInSet` string expression with collation") { - case class FindInSetTestCase[R](left: String, right: String, collation: String, result: R) - val testCases = Seq( - FindInSetTestCase("AB", "abc,b,ab,c,def", "UTF8_BINARY", 0), - FindInSetTestCase("b ", "abc,b,ab,c,def", "UTF8_BINARY_RTRIM", 2), - FindInSetTestCase("def", "abc,b,ab,c,def ", "UTF8_BINARY_RTRIM", 5), - FindInSetTestCase("C", "abc,b,ab,c,def", "UTF8_LCASE", 4), - FindInSetTestCase("C ", "abc,b,ab,c ,def", "UTF8_LCASE_RTRIM", 4), - FindInSetTestCase("d,ef", "abc,b,ab,c,def", "UNICODE", 0), - FindInSetTestCase(" def", "abc,b,ab,c,def", "UNICODE_RTRIM", 0), - FindInSetTestCase("i̇o", "ab,İo,12", "UNICODE_CI", 2), - FindInSetTestCase("İo", "ab,i̇o,12", "UNICODE_CI", 2), - FindInSetTestCase("İo", "ab,i̇o,12", "UNICODE_CI", 2), - FindInSetTestCase("a", "A ,B ,C", "UNICODE_CI_RTRIM", 1), - FindInSetTestCase(" a", "A ,B ,C", "UNICODE_CI_RTRIM", 0) - ) - testCases.foreach(t => { - // Unit test. - val left = Literal.create(t.left, StringType(t.collation)) - val right = Literal.create(t.right, StringType(t.collation)) - checkEvaluation(FindInSet(left, right), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select find_in_set('${t.left}', '${t.right}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(IntegerType)) - } - }) - } - - test("Support `StartsWith` string expression with collation") { - case class StartsWithTestCase[R](left: String, right: String, collation: String, result: R) - val testCases = Seq( - StartsWithTestCase("", "", "UTF8_BINARY", true), - StartsWithTestCase("", " ", "UTF8_BINARY_RTRIM", true), - StartsWithTestCase("abcde", "A", "UNICODE", false), - StartsWithTestCase("abcde", "a ", "UNICODE_RTRIM", true), - StartsWithTestCase("abcde", "FGH", "UTF8_LCASE", false), - StartsWithTestCase("abcde ", "FGH ", "UTF8_LCASE_RTRIM", false), - StartsWithTestCase("abcde", "ABC", "UNICODE_CI", true), - StartsWithTestCase("a b c de ", "A B C ", "UNICODE_CI_RTRIM", true), - StartsWithTestCase("abcde", "ABC", "UNICODE_CI", true), - StartsWithTestCase("Šuma", "šum", "SR_CI_AI", true), - StartsWithTestCase("Šuma", "šum", "SR", false) - ) - val unsupportedTestCase = StartsWithTestCase("abcde", "A", "UNICODE_AI", false) - testCases.foreach(t => { - // Unit test. - val left = Literal.create(t.left, StringType(t.collation)) - val right = Literal.create(t.right, StringType(t.collation)) - checkEvaluation(StartsWith(left, right), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select startswith('${t.left}', '${t.right}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(BooleanType)) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select startswith('${unsupportedTestCase.left}', '${unsupportedTestCase.right}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> "\"startswith('abcde' collate UNICODE_AI, 'A' collate UNICODE_AI)\"", - "paramIndex" -> "first", - "inputSql" -> "\"'abcde' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "startswith('abcde', 'A')", start = 7, stop = 30) - ) - } - } - - test("Support `StringTranslate` string expression with collation") { - case class StringTranslateTestCase[R]( - srcExpr: String, - matchingExpr: String, - replaceExpr: String, - collation: String, - result: R) - val testCases = Seq( - StringTranslateTestCase("Translate", "Rnlt", "12", "UTF8_BINARY", "Tra2sae"), - StringTranslateTestCase(" abc ", "abc", "123", "UTF8_BINARY_RTRIM", " 123 "), - StringTranslateTestCase("Translate", "Rnlt", "1234", "UTF8_LCASE", "41a2s3a4e"), - StringTranslateTestCase(" abc ", " AB", "123", "UTF8_LCASE_RTRIM", "123c1"), - StringTranslateTestCase("Translate", "Rn", "\u0000\u0000", "UNICODE", "Traslate"), - StringTranslateTestCase(" a b c ", "abc ", "1234", "UNICODE_RTRIM", "4142434"), - StringTranslateTestCase("Translate", "Rn", "1234", "UNICODE_CI", "T1a2slate"), - StringTranslateTestCase(" abc ", "AB ", "123", "UNICODE_CI_RTRIM", "312c3"), - StringTranslateTestCase("Translate", "Rn", "1234", "UNICODE_CI", "T1a2slate"), - StringTranslateTestCase("Êtèréêë", "te", "12", "AF_CI_AI", "212r222") - ) - val unsupportedTestCase = StringTranslateTestCase("ABC", "AB", "12", "UNICODE_AI", "12C") - testCases.foreach(t => { - // Unit test. - val srcExpr = Literal.create(t.srcExpr, StringType(t.collation)) - val matchingExpr = Literal.create(t.matchingExpr, StringType(t.collation)) - val replaceExpr = Literal.create(t.replaceExpr, StringType(t.collation)) - checkEvaluation(StringTranslate(srcExpr, matchingExpr, replaceExpr), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select translate('${t.srcExpr}', '${t.matchingExpr}', '${t.replaceExpr}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select translate('${unsupportedTestCase.srcExpr}', " + - s"'${unsupportedTestCase.matchingExpr}', '${unsupportedTestCase.replaceExpr}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> ("\"translate('ABC' collate UNICODE_AI, 'AB' collate UNICODE_AI, " + - "'12' collate UNICODE_AI)\""), - "paramIndex" -> "first", - "inputSql" -> "\"'ABC' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "translate('ABC', 'AB', '12')", start = 7, stop = 34) - ) - } - } - - test("Support `StringReplace` string expression with collation") { - case class StringReplaceTestCase[R]( - srcExpr: String, - searchExpr: String, - replaceExpr: String, - collation: String, - result: R) - val testCases = Seq( - StringReplaceTestCase("r世eplace", "pl", "123", "UTF8_BINARY", "r世e123ace"), - StringReplaceTestCase(" abc ", "b ", "x", "UTF8_BINARY_RTRIM", " abc "), - StringReplaceTestCase("repl世ace", "PL", "AB", "UTF8_LCASE", "reAB世ace"), - StringReplaceTestCase(" abc ", " AB", "123", "UTF8_LCASE_RTRIM", "123c "), - StringReplaceTestCase("abcdabcd", "bc", "", "UNICODE", "adad"), - StringReplaceTestCase(" abc ", "b ", "x", "UNICODE_RTRIM", " abc "), - StringReplaceTestCase("aBc世abc", "b", "12", "UNICODE_CI", "a12c世a12c"), - StringReplaceTestCase("abi̇o12i̇o", "İo", "yy", "UNICODE_CI", "abyy12yy"), - StringReplaceTestCase("abİo12i̇o", "i̇o", "xx", "UNICODE_CI", "abxx12xx"), - StringReplaceTestCase(" ABC ", "bc ", "123", "UNICODE_CI_RTRIM", " A123"), - StringReplaceTestCase("češalj", "eSal", "A", "SR_CI_AI", "čAj") - ) - val unsupportedTestCase = StringReplaceTestCase("abcde", "A", "B", "UNICODE_AI", "abcde") - testCases.foreach(t => { - // Unit test. - val srcExpr = Literal.create(t.srcExpr, StringType(t.collation)) - val searchExpr = Literal.create(t.searchExpr, StringType(t.collation)) - val replaceExpr = Literal.create(t.replaceExpr, StringType(t.collation)) - checkEvaluation(StringReplace(srcExpr, searchExpr, replaceExpr), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select replace('${t.srcExpr}', '${t.searchExpr}', '${t.replaceExpr}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select replace('${unsupportedTestCase.srcExpr}', '${unsupportedTestCase.searchExpr}', " + - s"'${unsupportedTestCase.replaceExpr}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> ("\"replace('abcde' collate UNICODE_AI, 'A' collate UNICODE_AI, " + - "'B' collate UNICODE_AI)\""), - "paramIndex" -> "first", - "inputSql" -> "\"'abcde' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "replace('abcde', 'A', 'B')", start = 7, stop = 32) - ) - } - } - - test("Support `EndsWith` string expression with collation") { - case class EndsWithTestCase[R](left: String, right: String, collation: String, result: R) - val testCases = Seq( - EndsWithTestCase("", "", "UTF8_BINARY", true), - EndsWithTestCase("", " ", "UTF8_BINARY_RTRIM", true), - EndsWithTestCase("abcde", "E", "UNICODE", false), - EndsWithTestCase("abcde ", "E ", "UNICODE_RTRIM", false), - EndsWithTestCase("abcde", "FGH", "UTF8_LCASE", false), - EndsWithTestCase("abcde ", "FGH ", "UTF8_LCASE_RTRIM", false), - EndsWithTestCase("abcde", "CDE", "UNICODE_CI", true), - EndsWithTestCase("abc d e ", "C D E", "UNICODE_CI_RTRIM", true), - EndsWithTestCase("abcde", "CDE", "UNICODE_CI", true), - EndsWithTestCase("xnigħat", "għat", "MT", true), - // The following two test cases showcase different behavior based on collation. - EndsWithTestCase("xnigħat", "ħat", "MT_CI", false), - EndsWithTestCase("muljavo", "javo", "SR_CI", true), - EndsWithTestCase("xnigħat", "GĦat", "MT_CI", true), - EndsWithTestCase("xnigħat", "Għat", "MT_CI", true) - ) - val unsupportedTestCase = EndsWithTestCase("abcde", "A", "UNICODE_AI", false) - testCases.foreach(t => { - // Unit test. - val left = Literal.create(t.left, StringType(t.collation)) - val right = Literal.create(t.right, StringType(t.collation)) - checkEvaluation(EndsWith(left, right), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select endswith('${t.left}', '${t.right}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(BooleanType)) - } - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select endswith('${unsupportedTestCase.left}', '${unsupportedTestCase.right}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> "\"endswith('abcde' collate UNICODE_AI, 'A' collate UNICODE_AI)\"", - "paramIndex" -> "first", - "inputSql" -> "\"'abcde' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "endswith('abcde', 'A')", start = 7, stop = 28) - ) - } - }) - } - - test("Support `StringRepeat` string expression with collation") { - case class StringRepeatTestCase[R](str: String, times: Integer, collation: String, result: R) - val testCases = Seq( - StringRepeatTestCase("", 1, "UTF8_BINARY", ""), - StringRepeatTestCase(" ", 1, "UTF8_BINARY_RTRIM", " "), - StringRepeatTestCase("a", 0, "UNICODE", ""), - StringRepeatTestCase("a", 0, "UNICODE_RTRIM", ""), - StringRepeatTestCase("XY", 3, "UTF8_LCASE", "XYXYXY"), - StringRepeatTestCase("XY ", 3, "UTF8_LCASE_RTRIM", "XY XY XY "), - StringRepeatTestCase("123", 2, "UNICODE_CI", "123123"), - StringRepeatTestCase("123 ", 2, "UNICODE_CI_RTRIM", "123 123 ") - ) - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType(t.collation)) - val times = Literal.create(t.times, IntegerType) - checkEvaluation(StringRepeat(str, times), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select repeat('${t.str}', ${t.times})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Ascii` string expression with collation") { - case class AsciiTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - AsciiTestCase("a", "UTF8_BINARY", 97), - AsciiTestCase("a ", "UTF8_BINARY_RTRIM", 97), - AsciiTestCase("B", "UTF8_LCASE", 66), - AsciiTestCase("B ", "UTF8_LCASE_RTRIM", 66), - AsciiTestCase("#", "UNICODE", 35), - AsciiTestCase("# ", "UNICODE_RTRIM", 35), - AsciiTestCase("!", "UNICODE_CI", 33), - AsciiTestCase("! ", "UNICODE_CI_RTRIM", 33) - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Ascii(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select ascii('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(IntegerType)) - } - }) - } - - test("Support `Chr` string expression with collation") { - case class ChrTestCase[R](input: Long, collation: String, result: R) - val testCases = Seq( - ChrTestCase(65, "UTF8_BINARY", "A"), - ChrTestCase(65, "UTF8_BINARY_RTRIM", "A"), - ChrTestCase(66, "UTF8_LCASE", "B"), - ChrTestCase(66, "UTF8_LCASE_RTRIM", "B"), - ChrTestCase(97, "UNICODE", "a"), - ChrTestCase(97, "UNICODE_RTRIM", "a"), - ChrTestCase(98, "UNICODE_CI", "b"), - ChrTestCase(98, "UNICODE_CI_RTRIM", "b") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Chr(Literal(t.input)), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select chr(${t.input})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `UnBase64` string expression with collation") { - case class UnBase64TestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - UnBase64TestCase("QUJD", "UTF8_BINARY", Array(65, 66, 67)), - UnBase64TestCase("QUJD", "UTF8_BINARY_RTRIM", Array(65, 66, 67)), - UnBase64TestCase("eHl6", "UTF8_LCASE", Array(120, 121, 122)), - UnBase64TestCase("eHl6", "UTF8_LCASE_RTRIM", Array(120, 121, 122)), - UnBase64TestCase("IyMj", "UNICODE", Array(35, 35, 35)), - UnBase64TestCase("IyMj", "UNICODE_RTRIM", Array(35, 35, 35)), - UnBase64TestCase("IQ==", "UNICODE_CI", Array(33)), - UnBase64TestCase("IQ==", "UNICODE_CI_RTRIM", Array(33)) - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Base64(UnBase64(Literal.create(t.input, StringType(t.collation)))), t.input) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select unbase64('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(BinaryType)) - } - }) - } - - test("Support `Base64` string expression with collation") { - case class Base64TestCase[R](input: Array[Byte], collation: String, result: R) - val testCases = Seq( - Base64TestCase(Array(65, 66, 67), "UTF8_BINARY", "QUJD"), - Base64TestCase(Array(65, 66, 67), "UTF8_BINARY_RTRIM", "QUJD"), - Base64TestCase(Array(120, 121, 122), "UTF8_LCASE", "eHl6"), - Base64TestCase(Array(120, 121, 122), "UTF8_LCASE_RTRIM", "eHl6"), - Base64TestCase(Array(35, 35, 35), "UNICODE", "IyMj"), - Base64TestCase(Array(35, 35, 35), "UNICODE_RTRIM", "IyMj"), - Base64TestCase(Array(33), "UNICODE_CI", "IQ=="), - Base64TestCase(Array(33), "UNICODE_CI_RTRIM", "IQ==") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Base64(Literal.create(t.input, BinaryType)), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val str = new String(t.input.map(_.toChar)) - val query = s"select base64('$str')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `FormatNumber` string expression with collation") { - case class FormatNumberTestCase[R](x: Double, d: String, collation: String, r: R) - val testCases = Seq( - FormatNumberTestCase(123.123, "###.###", "UTF8_BINARY", "123.123"), - FormatNumberTestCase(123.123, "###.###", "UTF8_BINARY_RTRIM", "123.123"), - FormatNumberTestCase(99.99, "##.##", "UTF8_LCASE", "99.99"), - FormatNumberTestCase(99.99, "##.##", "UTF8_LCASE_RTRIM", "99.99"), - FormatNumberTestCase(123.123, "###.###", "UNICODE", "123.123"), - FormatNumberTestCase(123.123, "###.###", "UNICODE_RTRIM", "123.123"), - FormatNumberTestCase(99.99, "##.##", "UNICODE_CI", "99.99"), - FormatNumberTestCase(99.99, "##.##", "UNICODE_CI_RTRIM", "99.99"), - FormatNumberTestCase(99.99, "##.##", "UNICODE_CI", "99.99"), - FormatNumberTestCase(99.999, "##.###", "AF_CI_AI", "99.999") - ) - testCases.foreach(t => { - // Unit test. - val x = Literal(t.x) - val d = Literal.create(t.d, StringType(t.collation)) - checkEvaluation(FormatNumber(x, d), t.r) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select format_number(${t.x}, '${t.d}')" - checkAnswer(sql(query), Row(t.r)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Decode` string expression with collation") { - case class DecodeTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - DecodeTestCase("a", "UTF8_BINARY", "a"), - DecodeTestCase("a", "UTF8_BINARY_RTRIM", "a"), - DecodeTestCase("A", "UTF8_LCASE", "A"), - DecodeTestCase("A", "UTF8_LCASE_RTRIM", "A"), - DecodeTestCase("b", "UNICODE", "b"), - DecodeTestCase("b", "UNICODE_RTRIM", "b"), - DecodeTestCase("B", "UNICODE_CI", "B"), - DecodeTestCase("B", "UNICODE_CI_RTRIM", "B") - ) - testCases.foreach(t => { - // Unit test. - val input = Literal.create(t.input, StringType(t.collation)) - val default = Literal.create("default", StringType(t.collation)) - val params = Seq(Literal(1), Literal(1), input, default) - checkEvaluation(Decode(params, Decode.createExpr(params)), t.input) - val encoding = Literal.create("UTF-8", StringType(t.collation)) - val encodeExpr = Encode(Literal.create(t.input, StringType(t.collation)), encoding) - checkEvaluation(StringDecode(encodeExpr, encoding), t.input) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val queryDecode = s"select decode(1, 1, '${t.input}', 'default')" - checkAnswer(sql(queryDecode), Row(t.result)) - assert(sql(queryDecode).schema.fields.head.dataType.sameType(StringType(t.collation))) - val queryStrDecode = s"select decode(encode('${t.input}', 'utf-8'), 'utf-8')" - checkAnswer(sql(queryStrDecode), Row(t.result)) - assert(sql(queryStrDecode).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Encode` string expression with collation") { - case class EncodeTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - EncodeTestCase("a", "UTF8_BINARY", Array(97)), - EncodeTestCase("a ", "UTF8_BINARY_RTRIM", Array(97, 32)), - EncodeTestCase("A", "UTF8_LCASE", Array(65)), - EncodeTestCase("A ", "UTF8_LCASE_RTRIM", Array(65, 32)), - EncodeTestCase("b", "UNICODE", Array(98)), - EncodeTestCase("b ", "UNICODE_RTRIM", Array(98, 32)), - EncodeTestCase("B", "UNICODE_CI", Array(66)), - EncodeTestCase("B ", "UNICODE_CI_RTRIM", Array(66, 32)) - ) - testCases.foreach(t => { - // Unit test. - val encoding = Literal.create("UTF-8", StringType(t.collation)) - val encodeExpr = Encode(Literal.create(t.input, StringType(t.collation)), encoding) - checkEvaluation(StringDecode(encodeExpr, encoding), t.input) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select encode('${t.input}', 'utf-8')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(BinaryType)) - } - }) - } - - test("Support `ToBinary` string expression with collation") { - case class ToBinaryTestCase[R](expr: String, format: String, collation: String, result: R) - val testCases = Seq( - ToBinaryTestCase("a", "utf-8", "UTF8_BINARY", Array(97)), - ToBinaryTestCase("a ", "utf-8", "UTF8_BINARY_RTRIM", Array(97, 32)), - ToBinaryTestCase("A", "utf-8", "UTF8_LCASE", Array(65)), - ToBinaryTestCase("A ", "utf-8", "UTF8_LCASE_RTRIM", Array(65, 32)), - ToBinaryTestCase("b", "utf-8", "UNICODE", Array(98)), - ToBinaryTestCase("b ", "utf-8", "UNICODE_RTRIM", Array(98, 32)), - ToBinaryTestCase("B", "utf-8", "UNICODE_CI", Array(66)), - ToBinaryTestCase("B ", "utf-8", "UNICODE_CI_RTRIM", Array(66, 32)) - ) - testCases.foreach(t => { - // Unit test. - val expr = Literal.create(t.expr, StringType(t.collation)) - val format = Literal.create(t.format, StringType(t.collation)) - checkEvaluation(StringDecode(ToBinary(expr, Some(format)), format), t.expr) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select to_binary('${t.expr}', 'utf-8')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(BinaryType)) - } - }) - } - - test("Support `Sentences` string expression with collation") { - case class SentencesTestCase[R](str: String, collation: String, result: R) - val testCases = Seq( - SentencesTestCase( - "Hello, world! Nice day.", - "UTF8_BINARY", - Seq(Seq("Hello", "world"), Seq("Nice", "day")) - ), - SentencesTestCase( - "Something else. Nothing here.", - "UTF8_LCASE", - Seq(Seq("Something", "else"), Seq("Nothing", "here")) - ), - SentencesTestCase( - "Hello, world! Nice day.", - "UNICODE", - Seq(Seq("Hello", "world"), Seq("Nice", "day")) - ), - SentencesTestCase( - "Something else. Nothing here.", - "UNICODE_CI", - Seq(Seq("Something", "else"), Seq("Nothing", "here")) - ), - SentencesTestCase( - "Hello, dinja! Ġurnata sabiħa.", - "MT_AI", - Seq(Seq("Hello", "dinja"), Seq("Ġurnata", "sabiħa")) - ) - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Sentences(Literal.create(t.str, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select sentences('${t.str}')" - checkAnswer(sql(query), Row(t.result)) - val expectedDataType = ArrayType(ArrayType(StringType(t.collation))) - assert(sql(query).schema.fields.head.dataType.sameType(expectedDataType)) - } - }) - } - - test("Support `Upper` string expression with collation") { - case class UpperTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - UpperTestCase("aBc", "UTF8_BINARY", "ABC"), - UpperTestCase("aBc ", "UTF8_BINARY_RTRIM", "ABC "), - UpperTestCase("aBc", "UTF8_LCASE", "ABC"), - UpperTestCase("aBc ", "UTF8_LCASE_RTRIM", "ABC "), - UpperTestCase("aBc", "UNICODE", "ABC"), - UpperTestCase("aBc ", "UNICODE_RTRIM", "ABC "), - UpperTestCase("aBc", "UNICODE_CI", "ABC"), - UpperTestCase("aBc ", "UNICODE_CI_RTRIM", "ABC "), - UpperTestCase("aBc", "UNICODE_CI", "ABC"), - UpperTestCase("xnìgħat", "MT_CI_AI", "XNÌGĦAT") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Upper(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select upper('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Lower` string expression with collation") { - case class LowerTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - LowerTestCase("aBc", "UTF8_BINARY", "abc"), - LowerTestCase("aBc ", "UTF8_BINARY_RTRIM", "abc "), - LowerTestCase("aBc", "UTF8_LCASE", "abc"), - LowerTestCase("aBc ", "UTF8_LCASE_RTRIM", "abc "), - LowerTestCase("aBc", "UNICODE", "abc"), - LowerTestCase("aBc ", "UNICODE_RTRIM", "abc "), - LowerTestCase("aBc", "UNICODE_CI", "abc"), - LowerTestCase("aBc ", "UNICODE_CI_RTRIM", "abc "), - LowerTestCase("aBc", "UNICODE_CI", "abc"), - LowerTestCase("VeRGrÖßeRn", "DE_CI", "vergrößern") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Lower(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select lower('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `InitCap` string expression with collation") { - case class InitCapTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - InitCapTestCase("aBc ABc", "UTF8_BINARY", "Abc Abc"), - InitCapTestCase(" aBc ABc ", "UTF8_BINARY_RTRIM", " Abc Abc "), - InitCapTestCase("aBc ABc", "UTF8_LCASE", "Abc Abc"), - InitCapTestCase(" aBc ABc ", "UTF8_LCASE_RTRIM", " Abc Abc "), - InitCapTestCase("aBc ABc", "UNICODE", "Abc Abc"), - InitCapTestCase(" aBc ABc ", "UNICODE_RTRIM", " Abc Abc "), - InitCapTestCase("aBc ABc", "UNICODE_CI", "Abc Abc"), - InitCapTestCase(" aBc ABc ", "UNICODE_CI_RTRIM", " Abc Abc "), - InitCapTestCase("aBc ABc", "UNICODE_CI", "Abc Abc"), - InitCapTestCase("æØÅ ÆøÅ", "NO", "Æøå Æøå") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(InitCap(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select initcap('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Overlay` string expression with collation") { - case class OverlayTestCase[R]( - input: String, - replace: String, - pos: Integer, - len: Integer, - collation: String, - result: R) - val testCases = Seq( - OverlayTestCase("hello", " world", 6, -1, "UTF8_BINARY", "hello world"), - OverlayTestCase("hello ", " world ", 7, -1, "UTF8_BINARY_RTRIM", "hello world "), - OverlayTestCase("nice", " day", 5, -1, "UTF8_LCASE", "nice day"), - OverlayTestCase(" nice ", " day ", 7, -1, "UTF8_LCASE_RTRIM", " nice day "), - OverlayTestCase("A", "B", 1, -1, "UNICODE", "B"), - OverlayTestCase("A", " B ", 1, -1, "UNICODE_RTRIM", " B "), - OverlayTestCase("!", "!!!", 1, -1, "UNICODE_CI", "!!!"), - OverlayTestCase("!", " !!! ", 1, -1, "UNICODE_CI_RTRIM", " !!! ") - ) - testCases.foreach(t => { - // Unit test. - val input = Literal.create(t.input, StringType(t.collation)) - val replace = Literal.create(t.replace, StringType(t.collation)) - val pos = Literal.create(t.pos, IntegerType) - val len = Literal.create(t.len, IntegerType) - checkEvaluation(Overlay(input, replace, pos, len), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select overlay('${t.input}' placing '${t.replace}' from ${t.pos})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `FormatString` string expression with collation") { - case class FormatStringTestCase[R]( - format: String, - input: Seq[Any], - collation: String, - result: R) - val testCases = Seq( - FormatStringTestCase("%s%s", Seq("a", "b"), "UTF8_BINARY", "ab"), - FormatStringTestCase("%s%s", Seq("a", "b "), "UTF8_BINARY_RTRIM", "ab "), - FormatStringTestCase("%d", Seq(123), "UTF8_LCASE", "123"), - FormatStringTestCase("%d", Seq(123), "UTF8_LCASE_RTRIM", "123"), - FormatStringTestCase("%s%d", Seq("A", 0), "UNICODE", "A0"), - FormatStringTestCase("%s%d", Seq(" A ", 0), "UNICODE_RTRIM", " A 0"), - FormatStringTestCase("%s%s", Seq("Hello", "!!!"), "UNICODE_CI", "Hello!!!"), - FormatStringTestCase("%s%s", Seq(" Hello ", " !!! "), "UNICODE_CI_RTRIM", " Hello !!! "), - FormatStringTestCase("%s%s", Seq("Hello", "!!!"), "UNICODE_CI", "Hello!!!"), - FormatStringTestCase("%s%s", Seq("Storslått", ".?!"), "NN_AI", "Storslått.?!") - ) - testCases.foreach(t => { - // Unit test. - val format = Literal.create(t.format, StringType(t.collation)) - val arguments = t.input.map { - case s: String => Literal.create(s, StringType(t.collation)) - case i: Integer => Literal.create(i, IntegerType) - } - checkEvaluation(FormatString(format +: arguments: _*), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val args = t.input - .map { - case s: String => s"'$s'" - case other => other.toString - } - .mkString(", ") - val query = s"select format_string('${t.format}', $args)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `SoundEx` string expression with collation") { - case class SoundExTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - SoundExTestCase("A", "UTF8_BINARY", "A000"), - SoundExTestCase("A", "UTF8_BINARY_RTRIM", "A000"), - SoundExTestCase("!", "UTF8_LCASE", "!"), - SoundExTestCase("!", "UTF8_LCASE_RTRIM", "!"), - SoundExTestCase("$", "UNICODE", "$"), - SoundExTestCase("$", "UNICODE_RTRIM", "$"), - SoundExTestCase("X", "UNICODE_CI", "X000"), - SoundExTestCase("X", "UNICODE_CI_RTRIM", "X000"), - SoundExTestCase("X", "UNICODE_CI", "X000"), - SoundExTestCase("ß", "DE", "ß") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(SoundEx(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select soundex('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Length` string expression with collation") { - case class LengthTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - LengthTestCase("", "UTF8_BINARY", 0), - LengthTestCase(" ", "UTF8_BINARY_RTRIM", 1), - LengthTestCase("abc", "UTF8_LCASE", 3), - LengthTestCase("abc ", "UTF8_LCASE_RTRIM", 4), - LengthTestCase("hello", "UNICODE", 5), - LengthTestCase("hello ", "UNICODE_RTRIM", 6), - LengthTestCase("ff", "UNICODE_CI", 1), - LengthTestCase("ff ", "UNICODE_CI_RTRIM", 2), - LengthTestCase("ff", "UNICODE_CI", 1), - LengthTestCase("groß", "DE_CI_AI", 4), - LengthTestCase("gross", "DE_AI", 5) - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Length(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select length('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(IntegerType)) - } - }) - } - - test("Support `BitLength` string expression with collation") { - case class BitLengthTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - BitLengthTestCase("", "UTF8_BINARY", 0), - BitLengthTestCase(" ", "UTF8_BINARY_RTRIM", 8), - BitLengthTestCase("abc", "UTF8_LCASE", 24), - BitLengthTestCase("abc ", "UTF8_LCASE_RTRIM", 32), - BitLengthTestCase("hello", "UNICODE", 40), - BitLengthTestCase("hello ", "UNICODE_RTRIM", 48), - BitLengthTestCase("ff", "UNICODE_CI", 24), - BitLengthTestCase("ff ", "UNICODE_CI_RTRIM", 32), - BitLengthTestCase("ff", "UNICODE_CI", 24), - BitLengthTestCase("GROß", "DE", 40) - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(BitLength(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select bit_length('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(IntegerType)) - } - }) - } - - test("Support `OctetLength` string expression with collation") { - case class OctetLengthTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - OctetLengthTestCase("", "UTF8_BINARY", 0), - OctetLengthTestCase(" ", "UTF8_BINARY_RTRIM", 1), - OctetLengthTestCase("abc", "UTF8_LCASE", 3), - OctetLengthTestCase("abc ", "UTF8_LCASE_RTRIM", 4), - OctetLengthTestCase("hello", "UNICODE", 5), - OctetLengthTestCase("hello ", "UNICODE_RTRIM", 6), - OctetLengthTestCase("ff", "UNICODE_CI", 3), - OctetLengthTestCase("ff ", "UNICODE_CI_RTRIM", 4) - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(OctetLength(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select octet_length('${t.input}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(IntegerType)) - } - }) - } - - test("Support `Luhncheck` string expression with collation") { - case class LuhncheckTestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - LuhncheckTestCase("123", "UTF8_BINARY", false), - LuhncheckTestCase("123", "UTF8_BINARY_RTRIM", false), - LuhncheckTestCase("000", "UTF8_LCASE", true), - LuhncheckTestCase("000", "UTF8_LCASE_RTRIM", true), - LuhncheckTestCase("111", "UNICODE", false), - LuhncheckTestCase("111", "UNICODE_RTRIM", false), - LuhncheckTestCase("222", "UNICODE_CI", false), - LuhncheckTestCase("222", "UNICODE_CI_RTRIM", false) - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(Luhncheck(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select luhn_check(${t.input})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(BooleanType)) - } - }) - } - - test("Support `Levenshtein` string expression with collation") { - case class LevenshteinTestCase[R]( - left: String, - right: String, - collation: String, - threshold: Option[Integer], - result: R) - val testCases = Seq( - LevenshteinTestCase("kitten", "sitTing", "UTF8_BINARY", None, 4), - LevenshteinTestCase("kitten", "sitTing ", "UTF8_BINARY_RTRIM", None, 6), - LevenshteinTestCase("kitten", "sitTing", "UTF8_LCASE", None, 4), - LevenshteinTestCase("kitten", "sitTing ", "UTF8_LCASE", None, 6), - LevenshteinTestCase("kitten", "sitTing", "UNICODE", Some(3), -1), - LevenshteinTestCase("kitten", "sitTing ", "UNICODE_RTRIM", Some(3), -1), - LevenshteinTestCase("kitten", "sitTing", "UNICODE_CI", Some(3), -1), - LevenshteinTestCase("kitten ", "sitTing ", "UNICODE_CI_RTRIM", Some(3), -1), - LevenshteinTestCase("kitten", "sitTing", "UNICODE_CI", Some(3), -1), - // Levenshtein function is currently not collation-aware (not considering case or accent). - LevenshteinTestCase("gr", "GR", "UNICODE_CI_AI", None, 2), - LevenshteinTestCase("groß", "Größer", "UNICODE_CI_AI", None, 4) - ) - testCases.foreach(t => { - // Unit test. - val left = Literal.create(t.left, StringType(t.collation)) - val right = Literal.create(t.right, StringType(t.collation)) - val threshold = t.threshold.map(Literal(_)) - checkEvaluation(Levenshtein(left, right, threshold), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val th = if (t.threshold.isDefined) s", ${t.threshold.get}" else "" - val query = s"select levenshtein('${t.left}', '${t.right}'$th)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(IntegerType)) - } - }) - } - - test("Support `IsValidUTF8` string expression with collation") { - case class IsValidUTF8TestCase[R](input: Any, collation: String, result: R) - val testCases = Seq( - IsValidUTF8TestCase(null, "UTF8_BINARY", null), - IsValidUTF8TestCase(null, "UTF8_BINARY_RTRIM", null), - IsValidUTF8TestCase("", "UTF8_LCASE", true), - IsValidUTF8TestCase("", "UTF8_LCASE_RTRIM", true), - IsValidUTF8TestCase("abc", "UNICODE", true), - IsValidUTF8TestCase("abc", "UNICODE_RTRIM", true), - IsValidUTF8TestCase("hello", "UNICODE_CI", true), - IsValidUTF8TestCase("hello", "UNICODE_CI_RTRIM", true), - IsValidUTF8TestCase("hello", "UNICODE_CI", true), - IsValidUTF8TestCase("ćao", "SR_CI_AI", true) - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(IsValidUTF8(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val str = if (t.input == null) "null" else s"'${t.input}'" - val query = s"select is_valid_utf8($str)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(BooleanType)) - } - }) - } - - test("Support `MakeValidUTF8` string expression with collation") { - case class MakeValidUTF8TestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - MakeValidUTF8TestCase(null, "UTF8_BINARY", null), - MakeValidUTF8TestCase(null, "UTF8_BINARY_RTRIM", null), - MakeValidUTF8TestCase("", "UTF8_LCASE", ""), - MakeValidUTF8TestCase("", "UTF8_LCASE_RTRIM", ""), - MakeValidUTF8TestCase("abc", "UNICODE", "abc"), - MakeValidUTF8TestCase("abc", "UNICODE_RTRIM", "abc"), - MakeValidUTF8TestCase("hello", "UNICODE_CI", "hello"), - MakeValidUTF8TestCase("hello", "UNICODE_CI_RTRIM", "hello") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(MakeValidUTF8(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val str = if (t.input == null) "null" else s"'${t.input}'" - val query = s"select make_valid_utf8($str)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `ValidateUTF8` string expression with collation") { - case class ValidateUTF8TestCase[R](input: String, collation: String, result: R) - val testCases = Seq( - ValidateUTF8TestCase(null, "UTF8_BINARY", null), - ValidateUTF8TestCase(null, "UTF8_BINARY_RTRIM", null), - ValidateUTF8TestCase("", "UTF8_LCASE", ""), - ValidateUTF8TestCase("", "UTF8_LCASE_RTRIM", ""), - ValidateUTF8TestCase("abc", "UNICODE", "abc"), - ValidateUTF8TestCase("abc", "UNICODE_RTRIM", "abc"), - ValidateUTF8TestCase("hello", "UNICODE_CI", "hello"), - ValidateUTF8TestCase("hello", "UNICODE_CI_RTRIM", "hello") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(ValidateUTF8(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val str = if (t.input == null) "null" else s"'${t.input}'" - val query = s"select validate_utf8($str)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `TryValidateUTF8` string expression with collation") { - case class ValidateUTF8TestCase(input: String, collation: String, result: Any) - val testCases = Seq( - ValidateUTF8TestCase(null, "UTF8_BINARY", null), - ValidateUTF8TestCase(null, "UTF8_BINARY_RTRIM", null), - ValidateUTF8TestCase("", "UTF8_LCASE", ""), - ValidateUTF8TestCase("", "UTF8_LCASE_RTRIM", ""), - ValidateUTF8TestCase("abc", "UNICODE", "abc"), - ValidateUTF8TestCase("abc", "UNICODE_RTRIM", "abc"), - ValidateUTF8TestCase("hello", "UNICODE_CI", "hello"), - ValidateUTF8TestCase("hello", "UNICODE_CI_RTRIM", "hello") - ) - testCases.foreach(t => { - // Unit test. - checkEvaluation(TryValidateUTF8(Literal.create(t.input, StringType(t.collation))), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val str = if (t.input == null) "null" else s"'${t.input}'" - val query = s"select try_validate_utf8($str)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Substring` string expression with collation") { - case class SubstringTestCase[R]( - str: String, - pos: Integer, - len: Option[Integer], - collation: String, - result: R) - val testCases = Seq( - SubstringTestCase("example", 1, Some(100), "UTF8_LCASE", "example"), - SubstringTestCase("example ", 1, Some(100), "UTF8_LCASE_RTRIM", "example "), - SubstringTestCase("example", 2, Some(2), "UTF8_BINARY", "xa"), - SubstringTestCase("example", 0, Some(0), "UNICODE", ""), - SubstringTestCase("example", 0, Some(0), "UNICODE_RTRIM", ""), - SubstringTestCase("example", -3, Some(2), "UNICODE_CI", "pl"), - SubstringTestCase("example ", -3, Some(2), "UNICODE_CI_RTRIM", "le"), - SubstringTestCase(" a世a ", 2, Some(3), "UTF8_LCASE", "a世a"), - SubstringTestCase("", 1, Some(1), "UTF8_LCASE", ""), - SubstringTestCase("", 1, Some(1), "UNICODE", ""), - SubstringTestCase(null, 1, None, "UTF8_BINARY", null), - SubstringTestCase(null, 1, Some(1), "UNICODE_CI", null), - SubstringTestCase(null, null, Some(null), "UTF8_BINARY", null), - SubstringTestCase(null, null, None, "UNICODE_CI", null), - SubstringTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", null, None, "UTF8_BINARY", null), - SubstringTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", null, None, "UTF8_BINARY_RTRIM", null), - SubstringTestCase("", null, None, "UNICODE_CI", null), - SubstringTestCase("", null, None, "UNICODE_CI", null), - SubstringTestCase("xnigħat", 4, Some(2), "MT_CI_AI", "għ") - ) - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType(t.collation)) - val pos = Literal.create(t.pos, IntegerType) - val len = Literal.create(t.len.getOrElse(Integer.MAX_VALUE), IntegerType) - checkEvaluation(Substring(str, pos, len), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val input = if (t.str == null) "null" else s"'${t.str}'" - val length = if (t.len.isDefined) s", ${t.len.get}" else "" - val query = s"select substring($input, ${t.pos}$length)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Left` string expression with collation") { - case class LeftTestCase[R](str: String, len: Integer, collation: String, result: R) - val testCases = Seq( - LeftTestCase(null, null, "UTF8_BINARY", null), - LeftTestCase(null, null, "UTF8_BINARY_RTRIM", null), - LeftTestCase(" a世a ", 3, "UTF8_LCASE", " a世"), - LeftTestCase(" a世a ", 3, "UTF8_LCASE_RTRIM", " a世"), - LeftTestCase("", 1, "UNICODE", ""), - LeftTestCase("", 1, "UNICODE_RTRIM", ""), - LeftTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", 3, "UNICODE", "ÀÃÂ"), - LeftTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", 3, "UNICODE_RTRIM", "ÀÃÂ"), - LeftTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", 3, "UNICODE", "ÀÃÂ"), - LeftTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", 7, "NO_AI", "ÀÃÂĀĂȦÄ") - ) - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType) - val len = Literal.create(t.len, IntegerType) - checkEvaluation(Left(str, len), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val input = if (t.str == null) "null" else s"'${t.str}'" - val query = s"select left($input, ${t.len})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `Right` string expression with collation") { - case class RightTestCase[R](str: String, len: Integer, collation: String, result: R) - val testCases = Seq( - RightTestCase(null, null, "UTF8_BINARY", null), - RightTestCase(null, null, "UTF8_BINARY_RTRIM", null), - RightTestCase(" a世a ", 3, "UTF8_LCASE", "世a "), - RightTestCase(" a世a ", 3, "UTF8_LCASE_RTRIM", "世a "), - RightTestCase("", 1, "UNICODE", ""), - RightTestCase("", 1, "UNICODE_RTRIM", ""), - RightTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", 3, "UNICODE", "ǢǼÆ"), - RightTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", 3, "UNICODE_RTRIM", "ǢǼÆ"), - RightTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", 3, "UNICODE", "ǢǼÆ"), - RightTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", 5, "NO_CI_AI", "ȻȻǢǼÆ") - ) - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType) - val len = Literal.create(t.len, IntegerType) - checkEvaluation(Right(str, len), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val input = if (t.str == null) "null" else s"'${t.str}'" - val query = s"select right($input, ${t.len})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `StringRPad` string expression with collation") { - case class StringRPadTestCase[R]( - str: String, - len: Integer, - pad: String, - collation: String, - result: R) - val testCases = Seq( - StringRPadTestCase("", 5, " ", "UTF8_BINARY", " "), - StringRPadTestCase("", 5, " ", "UTF8_BINARY_RTRIM", " "), - StringRPadTestCase("abc", 5, " ", "UNICODE", "abc "), - StringRPadTestCase("ab c ", 5, " ", "UNICODE_RTRIM", "ab c "), - StringRPadTestCase("Hello", 7, "Wörld", "UTF8_LCASE", "HelloWö"), - StringRPadTestCase("1234567890", 5, "aaaAAa", "UNICODE_CI", "12345"), - StringRPadTestCase("12 34567890", 5, "aaaAAa", "UNICODE_CI_RTRIM", "12 34"), - StringRPadTestCase("aaAA", 2, " ", "UTF8_BINARY", "aa"), - StringRPadTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ℀℃", 2, "1", "UTF8_LCASE", "ÀÃ"), - StringRPadTestCase("ĂȦÄäåäá", 20, "ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", "UNICODE", "ĂȦÄäåäáÀÃÂĀĂȦÄäåäáâã"), - StringRPadTestCase("ĂȦÄäåäá", 20, "ÀÃÂĀĂȦÄäåäáâãȻȻ", "UNICODE_RTRIM", "ĂȦÄäåäáÀÃÂĀĂȦÄäåäáâã"), - StringRPadTestCase("aȦÄä", 8, "a1", "UNICODE_CI", "aȦÄäa1a1"), - StringRPadTestCase("aȦÄä", 8, "a1", "UNICODE_CI_RTRIM", "aȦÄäa1a1") - ) - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType(t.collation)) - val len = Literal.create(t.len, IntegerType) - val pad = Literal.create(t.pad, StringType(t.collation)) - checkEvaluation(StringRPad(str, len, pad), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select rpad('${t.str}', ${t.len}, '${t.pad}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `StringLPad` string expression with collation") { - case class StringLPadTestCase[R]( - str: String, - len: Integer, - pad: String, - collation: String, - result: R) - val testCases = Seq( - StringLPadTestCase("", 5, " ", "UTF8_BINARY", " "), - StringLPadTestCase("", 5, " ", "UTF8_BINARY_RTRIM", " "), - StringLPadTestCase("abc", 5, " ", "UNICODE", " abc"), - StringLPadTestCase("Hello", 7, "Wörld", "UTF8_LCASE", "WöHello"), - StringLPadTestCase("Hello", 7, "W örld", "UTF8_LCASE_RTRIM", "W Hello"), - StringLPadTestCase("1234567890", 5, "aaaAAa", "UNICODE_CI", "12345"), - StringLPadTestCase("aaAA", 2, " ", "UTF8_BINARY", "aa"), - StringLPadTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ℀℃", 2, "1", "UTF8_LCASE", "ÀÃ"), - StringLPadTestCase("ĂȦÄäåäá", 20, "ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", "UNICODE", "ÀÃÂĀĂȦÄäåäáâãĂȦÄäåäá"), - StringLPadTestCase("ĂȦÄäåäá", 20, "ÀÃÂĀĂȦÄäåäáâãȻȻ", "UNICODE_RTRIM", "ÀÃÂĀĂȦÄäåäáâãĂȦÄäåäá"), - StringLPadTestCase("aȦÄä", 8, "a1", "UNICODE_CI", "a1a1aȦÄä"), - StringLPadTestCase("aȦÄ ", 8, "a1", "UNICODE_CI_RTRIM", "a1a1aȦÄ ") - ) - testCases.foreach(t => { - // Unit test. - val str = Literal.create(t.str, StringType(t.collation)) - val len = Literal.create(t.len, IntegerType) - val pad = Literal.create(t.pad, StringType(t.collation)) - checkEvaluation(StringLPad(str, len, pad), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select lpad('${t.str}', ${t.len}, '${t.pad}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - - test("Support `StringLocate` string expression with collation") { - case class StringLocateTestCase[R]( - substr: String, - str: String, - start: Integer, - collation: String, - result: R) - val testCases = Seq( - StringLocateTestCase("aa", "aaads", 0, "UTF8_BINARY", 0), - StringLocateTestCase(" ", "", 1, "UTF8_BINARY_RTRIM", 1), - StringLocateTestCase(" abc ", " cdfg abc ", 1, "UTF8_BINARY_RTRIM", 12), - StringLocateTestCase("aa", "Aaads", 0, "UTF8_LCASE", 0), - StringLocateTestCase("界x", "test大千世界X大千世界", 1, "UTF8_LCASE", 8), - StringLocateTestCase("aBc", "abcabc", 4, "UTF8_LCASE", 4), - StringLocateTestCase("aa", "Aaads", 0, "UTF8_LCASE_RTRIM", 0), - StringLocateTestCase("界 ", "test大千世界X大千世界", 1, "UTF8_LCASE_RTRIM", 8), - StringLocateTestCase("aBc", "a bc abc ", 4, "UTF8_LCASE_RTRIM", 6), - StringLocateTestCase("aa", "Aaads", 0, "UNICODE", 0), - StringLocateTestCase("abC", "abCabC", 2, "UNICODE", 4), - StringLocateTestCase("aa", "Aaads", 0, "UNICODE_RTRIM", 0), - StringLocateTestCase("abC ", "ab C abC ", 2, "UNICODE_RTRIM", 6), - StringLocateTestCase("aa", "Aaads", 0, "UNICODE_CI", 0), - StringLocateTestCase("界x", "test大千世界X大千世界", 1, "UNICODE_CI", 8), - StringLocateTestCase("aa", "Aaads", 0, "UNICODE_CI_RTRIM", 0), - StringLocateTestCase(" 界", "test大千世界X大千世界", 1, "UNICODE_CI_RTRIM", 0), - StringLocateTestCase("oa", "TÖäöäoAoa", 1, "DE", 8), - StringLocateTestCase("oa", "TÖäöäoAoa", 1, "DE_CI", 6), - StringLocateTestCase("oa", "TÖäöäoAoa", 1, "DE_CI_AI", 2) - ) - val unsupportedTestCase = StringLocateTestCase("aa", "Aaads", 0, "UNICODE_AI", 1) - testCases.foreach(t => { - // Unit test. - val substr = Literal.create(t.substr, StringType(t.collation)) - val str = Literal.create(t.str, StringType(t.collation)) - val start = Literal.create(t.start, IntegerType) - checkEvaluation(StringLocate(substr, str, start), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val query = s"select locate('${t.substr}', '${t.str}', ${t.start})" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(IntegerType)) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val query = - s"select locate('${unsupportedTestCase.substr}', '${unsupportedTestCase.str}', " + - s"${unsupportedTestCase.start})" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> "\"locate('aa' collate UNICODE_AI, 'Aaads' collate UNICODE_AI, 0)\"", - "paramIndex" -> "first", - "inputSql" -> "\"'aa' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "locate('aa', 'Aaads', 0)", start = 7, stop = 30) - ) - } - } - - test("Support `StringTrimLeft` string expression with collation") { - case class StringTrimLeftTestCase[R]( - srcStr: String, - trimStr: Option[String], - collation: String, - result: R) - val testCases = Seq( - StringTrimLeftTestCase("xxasdxx", Some("x"), "UTF8_BINARY", "asdxx"), - StringTrimLeftTestCase(" xxasdxx", Some("x"), "UTF8_BINARY_RTRIM", " xxasdxx"), - StringTrimLeftTestCase(" xxasdxx", Some("x "), "UTF8_BINARY_RTRIM", "asdxx"), - StringTrimLeftTestCase(" xxasdxx ", Some("x "), "UTF8_BINARY_RTRIM", "asdxx "), - StringTrimLeftTestCase("xxasdxx", Some("X"), "UTF8_LCASE", "asdxx"), - StringTrimLeftTestCase("xxasdxx ", Some("X "), "UTF8_LCASE_RTRIM", "asdxx "), - StringTrimLeftTestCase("xxasdxx ", Some("X"), "UTF8_LCASE_RTRIM", "asdxx "), - StringTrimLeftTestCase(" xxasdxx ", Some("X "), "UTF8_LCASE_RTRIM", "asdxx "), - StringTrimLeftTestCase("xxasdxx", Some("y"), "UNICODE", "xxasdxx"), - StringTrimLeftTestCase("xxasdxx", Some("y"), "UNICODE_RTRIM", "xxasdxx"), - StringTrimLeftTestCase(" asd ", None, "UNICODE_RTRIM", "asd "), - StringTrimLeftTestCase(" asd ", None, "UNICODE_CI", "asd "), - StringTrimLeftTestCase(" asd ", Some("A"), "UNICODE_CI_RTRIM", " asd "), - StringTrimLeftTestCase(" asd ", None, "UNICODE_CI", "asd "), - StringTrimLeftTestCase("ćĆčČcCabCcČčĆć", Some("ć"), "SR", "ĆčČcCabCcČčĆć"), - StringTrimLeftTestCase("ćĆčČcCabCcČčĆć", Some("Ć"), "SR_CI", "čČcCabCcČčĆć"), - StringTrimLeftTestCase("ćĆčČcCabCcČčĆć", Some("ć"), "SR_CI_AI", "abCcČčĆć") - ) - val unsupportedTestCase = StringTrimLeftTestCase("xxasdxx", Some("x"), "UNICODE_AI", null) - testCases.foreach(t => { - // Unit test. - val srcStr = Literal.create(t.srcStr, StringType(t.collation)) - val trimStr = t.trimStr.map(Literal.create(_, StringType(t.collation))) - checkEvaluation(StringTrimLeft(srcStr, trimStr), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val trimString = if (t.trimStr.isDefined) s"'${t.trimStr.get}', " else "" - val query = s"select ltrim($trimString'${t.srcStr}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val trimString = s"'${unsupportedTestCase.trimStr.get}', " - val query = s"select ltrim($trimString'${unsupportedTestCase.srcStr}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> "\"TRIM(LEADING 'x' collate UNICODE_AI FROM 'xxasdxx' collate UNICODE_AI)\"", - "paramIndex" -> "first", - "inputSql" -> "\"'xxasdxx' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "ltrim('x', 'xxasdxx')", start = 7, stop = 27) - ) - } - } - - test("Support `StringTrimRight` string expression with collation") { - case class StringTrimRightTestCase[R]( - srcStr: String, - trimStr: Option[String], - collation: String, - result: R) - val testCases = Seq( - StringTrimRightTestCase(" xxasdxx", Some("x "), "UTF8_BINARY", " xxasd"), - StringTrimRightTestCase("xxasdxx ", Some("x "), "UTF8_BINARY_RTRIM", "xxasd"), - StringTrimRightTestCase("xxasdxx ", Some("x"), "UTF8_BINARY_RTRIM", "xxasd "), - StringTrimRightTestCase(" xxasdxx ", Some("x "), "UTF8_BINARY_RTRIM", " xxasd"), - StringTrimRightTestCase(" xxasdxx", Some("x"), "UTF8_BINARY_RTRIM", " xxasd"), - StringTrimRightTestCase("xxasdxx", Some("X"), "UTF8_LCASE", "xxasd"), - StringTrimRightTestCase("xxasdxx ", Some("X "), "UTF8_LCASE_RTRIM", "xxasd"), - StringTrimRightTestCase("xxasdxx ", Some("X"), "UTF8_LCASE_RTRIM", "xxasd "), - StringTrimRightTestCase(" xxasdxx ", Some("X "), "UTF8_LCASE_RTRIM", " xxasd"), - StringTrimRightTestCase(" xxasdxx", Some("x"), "UTF8_LCASE_RTRIM", " xxasd"), - StringTrimRightTestCase("xxasdxx", Some("y"), "UNICODE", "xxasdxx"), - StringTrimRightTestCase("xxasdxx", Some("y"), "UNICODE_RTRIM", "xxasdxx"), - StringTrimRightTestCase(" asd ", None, "UNICODE_RTRIM", " asd"), - StringTrimRightTestCase(" asd ", None, "UNICODE_CI", " asd"), - StringTrimRightTestCase(" asd ", Some("D"), "UNICODE_CI_RTRIM", " as "), - StringTrimRightTestCase(" asd ", None, "UNICODE_CI", " asd"), - StringTrimRightTestCase("ćĆčČcCabCcČčĆć", Some("ć"), "SR", "ćĆčČcCabCcČčĆ"), - StringTrimRightTestCase("ćĆčČcCabCcČčĆć", Some("Ć"), "SR_CI", "ćĆčČcCabCcČč"), - StringTrimRightTestCase("ćĆčČcCabCcČčĆć", Some("ć"), "SR_CI_AI", "ćĆčČcCab") - ) - val unsupportedTestCase = StringTrimRightTestCase("xxasdxx", Some("x"), "UNICODE_AI", "xxasd") - testCases.foreach(t => { - // Unit test. - val srcStr = Literal.create(t.srcStr, StringType(t.collation)) - val trimStr = t.trimStr.map(Literal.create(_, StringType(t.collation))) - checkEvaluation(StringTrimRight(srcStr, trimStr), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val trimString = if (t.trimStr.isDefined) s"'${t.trimStr.get}', " else "" - val query = s"select rtrim($trimString'${t.srcStr}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val trimString = s"'${unsupportedTestCase.trimStr.get}', " - val query = s"select rtrim($trimString'${unsupportedTestCase.srcStr}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> ("\"TRIM(TRAILING 'x' collate UNICODE_AI FROM 'xxasdxx'" + - " collate UNICODE_AI)\""), - "paramIndex" -> "first", - "inputSql" -> "\"'xxasdxx' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "rtrim('x', 'xxasdxx')", start = 7, stop = 27) - ) - } - } - - test("Support `StringTrim` string expression with collation") { - case class StringTrimTestCase[R]( - srcStr: String, - trimStr: Option[String], - collation: String, - result: R) - val testCases = Seq( - StringTrimTestCase("xxasdxx", Some("x"), "UTF8_BINARY", "asd"), - StringTrimTestCase("xxasdxx ", Some("x "), "UTF8_BINARY_RTRIM", "asd"), - StringTrimTestCase("xxasdxx ", Some("x"), "UTF8_BINARY_RTRIM", "asd "), - StringTrimTestCase(" xxasdxx ", Some("x "), "UTF8_BINARY_RTRIM", "asd"), - StringTrimTestCase(" xxasdxx", Some("x"), "UTF8_BINARY_RTRIM", " xxasd"), - StringTrimTestCase("xxasdxx", Some("X"), "UTF8_LCASE", "asd"), - StringTrimTestCase("xxasdxx ", Some("X "), "UTF8_LCASE_RTRIM", "asd"), - StringTrimTestCase("xxasdxx ", Some("X"), "UTF8_LCASE_RTRIM", "asd "), - StringTrimTestCase(" xxasdxx ", Some("X "), "UTF8_LCASE_RTRIM", "asd"), - StringTrimTestCase(" xxasdxx", Some("x"), "UTF8_LCASE_RTRIM", " xxasd"), - StringTrimTestCase("xxasdxx", Some("y"), "UNICODE", "xxasdxx"), - StringTrimTestCase("xxasdxx", Some("y"), "UNICODE_RTRIM", "xxasdxx"), - StringTrimTestCase(" asd ", None, "UNICODE_RTRIM", "asd"), - StringTrimTestCase(" asd ", None, "UNICODE_CI", "asd"), - StringTrimTestCase(" asd ", Some("D"), "UNICODE_CI_RTRIM", " as "), - StringTrimTestCase(" asd ", None, "UNICODE_CI", "asd"), - StringTrimTestCase("ćĆčČcCabCcČčĆć", Some("ć"), "SR", "ĆčČcCabCcČčĆ"), - StringTrimTestCase("ćĆčČcCabCcČčĆć", Some("Ć"), "SR_CI", "čČcCabCcČč"), - StringTrimTestCase("ćĆčČcCabCcČčĆć", Some("ć"), "SR_CI_AI", "ab"), - StringTrimTestCase(" ćĆčČcCabCcČčĆć ", None, "SR_CI_AI", "ćĆčČcCabCcČčĆć") - ) - val unsupportedTestCase = StringTrimTestCase("xxasdxx", Some("x"), "UNICODE_AI", "asd") - testCases.foreach(t => { - // Unit test. - val srcStr = Literal.create(t.srcStr, StringType(t.collation)) - val trimStr = t.trimStr.map(Literal.create(_, StringType(t.collation))) - checkEvaluation(StringTrim(srcStr, trimStr), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val trimString = if (t.trimStr.isDefined) s"'${t.trimStr.get}', " else "" - val query = s"select trim($trimString'${t.srcStr}')" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - // Test unsupported collation. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> unsupportedTestCase.collation) { - val trimString = s"'${unsupportedTestCase.trimStr.get}', " - val query = s"select trim($trimString'${unsupportedTestCase.srcStr}')" - checkError( - exception = intercept[AnalysisException] { - sql(query).collect() - }, - condition = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - sqlState = Some("42K09"), - parameters = Map( - "sqlExpr" -> "\"TRIM(BOTH 'x' collate UNICODE_AI FROM 'xxasdxx' collate UNICODE_AI)\"", - "paramIndex" -> "first", - "inputSql" -> "\"'xxasdxx' collate UNICODE_AI\"", - "inputType" -> "\"STRING COLLATE UNICODE_AI\"", - "requiredType" -> "\"STRING\""), - context = ExpectedContext(fragment = "trim('x', 'xxasdxx')", start = 7, stop = 26) - ) - } - } - - test("Support `StringTrimBoth` string expression with collation") { - case class StringTrimBothTestCase[R]( - srcStr: String, - trimStr: Option[String], - collation: String, - result: R) - val testCases = Seq( - StringTrimBothTestCase("xxasdxx", Some("x"), "UTF8_BINARY", "asd"), - StringTrimBothTestCase("xxasdxx ", Some("x "), "UTF8_BINARY_RTRIM", "asd"), - StringTrimBothTestCase("xxasdxx ", Some("x"), "UTF8_BINARY_RTRIM", "asd "), - StringTrimBothTestCase(" xxasdxx ", Some("x "), "UTF8_BINARY_RTRIM", "asd"), - StringTrimBothTestCase(" xxasdxx", Some("x"), "UTF8_BINARY_RTRIM", " xxasd"), - StringTrimBothTestCase("xxasdxx", Some("X"), "UTF8_LCASE", "asd"), - StringTrimBothTestCase("xxasdxx ", Some("X "), "UTF8_LCASE_RTRIM", "asd"), - StringTrimBothTestCase("xxasdxx ", Some("X"), "UTF8_LCASE_RTRIM", "asd "), - StringTrimBothTestCase(" xxasdxx ", Some("X "), "UTF8_LCASE_RTRIM", "asd"), - StringTrimBothTestCase(" xxasdxx", Some("x"), "UTF8_LCASE_RTRIM", " xxasd"), - StringTrimBothTestCase("xxasdxx", Some("y"), "UNICODE", "xxasdxx"), - StringTrimBothTestCase("xxasdxx", Some("y"), "UNICODE_RTRIM", "xxasdxx"), - StringTrimBothTestCase(" asd ", None, "UNICODE_RTRIM", "asd"), - StringTrimBothTestCase(" asd ", None, "UNICODE_CI", "asd"), - StringTrimBothTestCase(" asd ", Some("D"), "UNICODE_CI_RTRIM", " as "), - StringTrimBothTestCase(" asd ", None, "UNICODE_CI", "asd"), - StringTrimBothTestCase("ćĆčČcCabCcČčĆć", Some("ć"), "SR", "ĆčČcCabCcČčĆ"), - StringTrimBothTestCase("ćĆčČcCabCcČčĆć", Some("Ć"), "SR_CI", "čČcCabCcČč"), - StringTrimBothTestCase("ćĆčČcCabCcČčĆć", Some("ć"), "SR_CI_AI", "ab"), - StringTrimBothTestCase(" ćĆčČcCabCcČčĆć ", None, "SR_CI_AI", "ćĆčČcCabCcČčĆć") - ) - testCases.foreach(t => { - // Unit test. - val srcStr = Literal.create(t.srcStr, StringType(t.collation)) - val trimStr = t.trimStr.map(Literal.create(_, StringType(t.collation))) - val replacement = StringTrim(srcStr, trimStr) - checkEvaluation(StringTrimBoth(srcStr, trimStr, replacement), t.result) - // E2E SQL test. - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> t.collation) { - val trimString = if (t.trimStr.isDefined) s", '${t.trimStr.get}'" else "" - val query = s"select btrim('${t.srcStr}'$trimString)" - checkAnswer(sql(query), Row(t.result)) - assert(sql(query).schema.fields.head.dataType.sameType(StringType(t.collation))) - } - }) - } - -} -// scalastyle:on nonascii - -class CollationStringExpressionsANSISuite extends CollationStringExpressionsSuite { - override protected def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.ANSI_ENABLED, true) - - // TODO: If needed, add more tests for other string expressions (with ANSI mode enabled) - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index a075198fb1493..f60b7afe816d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.jdk.CollectionConverters.MapHasAsJava -import org.apache.spark.{SparkException, SparkThrowable} +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.CollationFactory @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAg import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.functions.col -import org.apache.spark.sql.internal.{SqlApiConf, SQLConf} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, IntegerType, MapType, Metadata, MetadataBuilder, StringType, StructField, StructType} class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { @@ -134,20 +134,6 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { StringType("UTF8_LCASE_RTRIM")) } - test("collate function syntax with default collation set") { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UTF8_LCASE") { - assert( - sql(s"select collate('aaa', 'utf8_lcase')").schema(0).dataType == - StringType("UTF8_LCASE") - ) - assert(sql(s"select collate('aaa', 'UNICODE')").schema(0).dataType == StringType("UNICODE")) - assert( - sql(s"select collate('aaa', 'UNICODE_RTRIM')").schema(0).dataType == - StringType("UNICODE_RTRIM") - ) - } - } - test("collate function syntax invalid arg count") { Seq("'aaa','a','b'", "'aaa'", "", "'aaa'").foreach(args => { val paramCount = if (args == "") 0 else args.split(',').length.toString @@ -822,31 +808,12 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { Seq(Row(fullyQualifiedPrefix + "UTF8_BINARY")) ) - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkAnswer( - sql( - """EXECUTE IMMEDIATE stmtStr1 USING - | 'a' AS var1, - | 'b' AS var2;""".stripMargin), - Seq(Row(fullyQualifiedPrefix + "UNICODE")) - ) - } - checkAnswer( sql( """EXECUTE IMMEDIATE stmtStr2 USING | 'a' AS var1;""".stripMargin), Seq(Row(fullyQualifiedPrefix + "UNICODE")) ) - - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkAnswer( - sql( - """EXECUTE IMMEDIATE stmtStr2 USING - | 'a' AS var1;""".stripMargin), - Seq(Row(fullyQualifiedPrefix + "UNICODE")) - ) - } } test("SPARK-47210: Cast of default collated strings in IN expression") { @@ -1009,12 +976,6 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { "generation expression cannot contain non utf8 binary collated string type")) } - test("SPARK-47431: Default collation set to UNICODE, literal test") { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkAnswer(sql(s"SELECT collation('aa')"), Seq(Row(fullyQualifiedPrefix + "UNICODE"))) - } - } - test("Cast expression for collations") { checkAnswer( sql(s"SELECT collation(cast('a' as string collate utf8_lcase))"), @@ -1026,15 +987,6 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { checkAnswer(sql(s"SELECT cast(1 as string)"), Seq(Row("1"))) checkAnswer(sql(s"SELECT cast('A' as string)"), Seq(Row("A"))) - - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkAnswer( - sql(s"SELECT collation(cast(1 as string collate unicode))"), - Seq(Row(fullyQualifiedPrefix + "UNICODE"))) - checkAnswer(sql(s"SELECT cast(1 as string)"), Seq(Row("1"))) - checkAnswer(sql(s"SELECT collation(cast(1 as string))"), - Seq(Row(fullyQualifiedPrefix + "UNICODE"))) - } } test("cast using the dataframe api") { @@ -1922,12 +1874,10 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { HllSketchAggTestCase("UNICODE_CI_RTRIM", 3) ) testCases.foreach(t => { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.c) { - val q = "SELECT hll_sketch_estimate(hll_sketch_agg(col)) FROM " + - "VALUES ('a'), ('A'), ('b'), ('b'), ('c'), ('c ') tab(col)" - val df = sql(q) - checkAnswer(df, Seq(Row(t.result))) - } + val q = s"SELECT hll_sketch_estimate(hll_sketch_agg(col collate ${t.c})) FROM " + + "VALUES ('a'), ('A'), ('b'), ('b'), ('c'), ('c ') tab(col)" + val df = sql(q) + checkAnswer(df, Seq(Row(t.result))) }) } @@ -1957,10 +1907,8 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { withTable("tbl") { checkCacheTable(s"'a' COLLATE $collation") } - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collation) { - withTable("tbl") { - checkCacheTable("'a'") - } + withTable("tbl") { + checkCacheTable("'a'") } } } @@ -2114,28 +2062,4 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"CREATE TABLE t (c STRING COLLATE system.builtin.UTF8_LCASE)") } } - - test("flag for enabling session default collation") { - withSQLConf(SQLConf.DEFAULT_COLLATION_ENABLED.key -> "false") { - checkError( - exception = intercept[SparkThrowable] { - sql("SET COLLATION UNICODE_CI") - }, - condition = "INVALID_CONF_VALUE.DEFAULT_COLLATION_NOT_SUPPORTED", - sqlState = "22022", - parameters = Map("confValue" -> "UNICODE_CI", - "confName" -> "spark.sql.session.collation.default")) - - checkAnswer( - sql("SELECT 'a' = 'A'"), - Row(false)) - } - - withSQLConf(SQLConf.DEFAULT_COLLATION_ENABLED.key -> "true") { - sql("SET COLLATION UNICODE_CI") - checkAnswer( - sql("SELECT 'a' = 'A'"), - Row(true)) - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 2e91d60e4ba04..0df7f806272c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -469,12 +469,10 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { withTable(testTable) { sql(s"CREATE TABLE $testTable (num int) USING parquet") sql(s"INSERT INTO $testTable VALUES (1), (2), (3), (NULL)") - Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI").foreach(collation => - withSQLConf(SQLConf.DEFAULT_COLLATION.key -> collation) { - val query = s"SELECT num, SUBSTRING_INDEX('a_a_a', '_', num) as sub_str FROM $testTable" - checkAnswer(sql(query), Seq(Row(1, "a"), Row(2, "a_a"), Row(3, "a_a_a"), Row(null, null))) - } - ) + Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI").foreach(collation => { + val query = s"SELECT num, SUBSTRING_INDEX('a_a_a', '_', num) as sub_str FROM $testTable" + checkAnswer(sql(query), Seq(Row(1, "a"), Row(2, "a_a"), Row(3, "a_a_a"), Row(null, null))) + }) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSQLFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSQLFunctionsSuite.scala index 83ec8c8d1bafb..e21c3b8c49c1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSQLFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSQLFunctionsSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.collation import org.apache.spark.sql.{Column, Dataset, QueryTest} import org.apache.spark.sql.functions.{from_json, from_xml} -import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -36,25 +35,23 @@ class CollationSQLFunctionsSuite extends QueryTest with SharedSparkSession { assert(transformedSchema === expectedSchema) } - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE_CI_AI") { - Seq( - StringType, - StringType("UTF8_BINARY"), - StringType("UNICODE"), - StringType("UNICODE_CI_AI")).foreach { stringType => - val dataSchema = StructType(Seq(StructField("fieldName", stringType))) - val expectedSchema = StructType(Seq(StructField("result", dataSchema))) - - // JSON Test - val jsonData = Seq("""{"fieldName": "fieldValue"}""") - val jsonDataset = spark.createDataset(jsonData) - checkSchema(jsonDataset, from_json($"value", dataSchema), expectedSchema) - - // XML Test - val xmlData = Seq("fieldValue") - val xmlDataset = spark.createDataset(xmlData) - checkSchema(xmlDataset, from_xml($"value", dataSchema), expectedSchema) - } + Seq( + StringType, + StringType("UTF8_BINARY"), + StringType("UNICODE"), + StringType("UNICODE_CI_AI")).foreach { stringType => + val dataSchema = StructType(Seq(StructField("fieldName", stringType))) + val expectedSchema = StructType(Seq(StructField("result", dataSchema))) + + // JSON Test + val jsonData = Seq("""{"fieldName": "fieldValue"}""") + val jsonDataset = spark.createDataset(jsonData) + checkSchema(jsonDataset, from_json($"value", dataSchema), expectedSchema) + + // XML Test + val xmlData = Seq("fieldValue") + val xmlDataset = spark.createDataset(xmlData) + checkSchema(xmlDataset, from_xml($"value", dataSchema), expectedSchema) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala index 1c765745b12dc..9e1968022c744 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.collation import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.util.CollationFactory import org.apache.spark.sql.connector.DatasourceV2SQLBase -import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StringType @@ -31,28 +30,6 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi def testView: String = "test_view" protected val fullyQualifiedPrefix = s"${CollationFactory.CATALOG}.${CollationFactory.SCHEMA}." - def withSessionCollationAndTable(collation: String, testTables: String*)(f: => Unit): Unit = { - withTable(testTables: _*) { - withSessionCollation(collation) { - f - } - } - } - - def withSessionCollationAndView(collation: String, viewNames: String*)(f: => Unit): Unit = { - withView(viewNames: _*) { - withSessionCollation(collation) { - f - } - } - } - - def withSessionCollation(collation: String)(f: => Unit): Unit = { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collation) { - f - } - } - def assertTableColumnCollation( table: String, column: String, @@ -71,7 +48,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi // region DDL tests test("create/alter table") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { // create table with implicit collation sql(s"CREATE TABLE $testTable (c1 STRING) USING $dataSource") assertTableColumnCollation(testTable, "c1", "UTF8_BINARY") @@ -89,20 +66,65 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } test("create table with explicit collation") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UTF8_LCASE) USING $dataSource") assertTableColumnCollation(testTable, "c1", "UTF8_LCASE") } - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UNICODE) USING $dataSource") assertTableColumnCollation(testTable, "c1", "UNICODE") } } + test("create/alter table with table level collation") { + withTable(testTable) { + // create table with default table level collation and explicit collation for some columns + sql(s"CREATE TABLE $testTable " + + s"(c1 STRING, c2 STRING COLLATE SR, c3 STRING COLLATE UTF8_BINARY, c4 STRING, id INT) " + + s"USING $dataSource DEFAULT COLLATION UTF8_LCASE") + assertTableColumnCollation(testTable, "c1", "UTF8_LCASE") + assertTableColumnCollation(testTable, "c2", "SR") + assertTableColumnCollation(testTable, "c3", "UTF8_BINARY") + assertTableColumnCollation(testTable, "c4", "UTF8_LCASE") + + // alter table add column + sql(s"ALTER TABLE $testTable ADD COLUMN c5 STRING") + assertTableColumnCollation(testTable, "c5", "UTF8_LCASE") + + // alter table default collation should not affect existing columns + sql(s"ALTER TABLE $testTable DEFAULT COLLATION UNICODE") + assertTableColumnCollation(testTable, "c1", "UTF8_LCASE") + assertTableColumnCollation(testTable, "c2", "SR") + assertTableColumnCollation(testTable, "c3", "UTF8_BINARY") + assertTableColumnCollation(testTable, "c4", "UTF8_LCASE") + assertTableColumnCollation(testTable, "c5", "UTF8_LCASE") + + // alter table add column, where the new column should pick up new collation + sql(s"ALTER TABLE $testTable ADD COLUMN c6 STRING") + assertTableColumnCollation(testTable, "c6", "UNICODE") + + // alter table alter column with explicit collation change + sql(s"ALTER TABLE $testTable ALTER COLUMN c1 TYPE STRING COLLATE UNICODE_CI") + assertTableColumnCollation(testTable, "c1", "UNICODE_CI") + + // alter table add columns with explicit collation, check collation for each column + sql(s"ALTER TABLE $testTable ADD COLUMN c7 STRING COLLATE SR_CI_AI") + sql(s"ALTER TABLE $testTable ADD COLUMN c8 STRING COLLATE UTF8_BINARY") + assertTableColumnCollation(testTable, "c1", "UNICODE_CI") + assertTableColumnCollation(testTable, "c2", "SR") + assertTableColumnCollation(testTable, "c3", "UTF8_BINARY") + assertTableColumnCollation(testTable, "c4", "UTF8_LCASE") + assertTableColumnCollation(testTable, "c5", "UTF8_LCASE") + assertTableColumnCollation(testTable, "c6", "UNICODE") + assertTableColumnCollation(testTable, "c7", "SR_CI_AI") + assertTableColumnCollation(testTable, "c8", "UTF8_BINARY") + } + } + test("create table as select") { // literals in select do not pick up session collation - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s""" |CREATE TABLE $testTable USING $dataSource AS SELECT | 'a' AS c1, @@ -120,7 +142,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } // literals in inline table do not pick up session collation - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s""" |CREATE TABLE $testTable USING $dataSource AS |SELECT c1, c1 = 'A' as c2 FROM VALUES ('a'), ('A') AS vals(c1) @@ -130,14 +152,14 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } // cast in select does not pick up session collation - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s"CREATE TABLE $testTable USING $dataSource AS SELECT cast('a' AS STRING) AS c1") assertTableColumnCollation(testTable, "c1", "UTF8_BINARY") } } test("ctas with complex types") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s""" |CREATE TABLE $testTable USING $dataSource AS |SELECT @@ -156,7 +178,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } test("ctas with union") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s""" |CREATE TABLE $testTable USING $dataSource AS |SELECT 'a' = 'A' AS c1 @@ -167,7 +189,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi checkAnswer(sql(s"SELECT * FROM $testTable"), Seq(Row(false))) } - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s""" |CREATE TABLE $testTable USING $dataSource AS |SELECT 'a' = 'A' AS c1 @@ -180,7 +202,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } test("add column") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UTF8_LCASE) USING $dataSource") assertTableColumnCollation(testTable, "c1", "UTF8_LCASE") @@ -193,7 +215,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } test("inline table in CTAS") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s""" |CREATE TABLE $testTable |USING $dataSource @@ -211,7 +233,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi // since concat coercion happens after resolving default types this test // makes sure that we are correctly resolving the default string types // in subsequent analyzer iterations - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s""" |CREATE TABLE $testTable |USING $dataSource AS @@ -222,7 +244,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } // ELT is similar - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s""" |CREATE TABLE $testTable |USING $dataSource AS @@ -234,169 +256,12 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } // endregion - - // region DML tests - - test("literals with default collation") { - val sessionCollation = "UTF8_LCASE" - val sessionCollationFullyQualified = fullyQualifiedPrefix + sessionCollation - withSessionCollation(sessionCollation) { - - // literal without collation - checkAnswer(sql("SELECT COLLATION('a')"), Seq(Row(sessionCollationFullyQualified))) - - checkAnswer(sql("SELECT COLLATION(map('a', 'b')['a'])"), - Seq(Row(sessionCollationFullyQualified))) - - checkAnswer(sql("SELECT COLLATION(array('a')[0])"), Seq(Row(sessionCollationFullyQualified))) - - checkAnswer(sql("SELECT COLLATION(struct('a' as c)['c'])"), - Seq(Row(sessionCollationFullyQualified))) - } - } - - test("literals with explicit collation") { - val unicodeCollation = fullyQualifiedPrefix + "UNICODE" - withSessionCollation("UTF8_LCASE") { - checkAnswer(sql("SELECT COLLATION('a' collate unicode)"), Seq(Row(unicodeCollation))) - - checkAnswer( - sql("SELECT COLLATION(map('a', 'b' collate unicode)['a'])"), - Seq(Row(unicodeCollation))) - - checkAnswer(sql("SELECT COLLATION(array('a' collate unicode)[0])"), - Seq(Row(unicodeCollation))) - - checkAnswer( - sql("SELECT COLLATION(struct('a' collate unicode as c)['c'])"), - Seq(Row(unicodeCollation))) - } - } - - test("cast is aware of session collation") { - val sessionCollation = "UTF8_LCASE" - val sessionCollationFullyQualified = fullyQualifiedPrefix + sessionCollation - withSessionCollation(sessionCollation) { - checkAnswer(sql("SELECT COLLATION(cast('a' as STRING))"), - Seq(Row(sessionCollationFullyQualified))) - - checkAnswer( - sql("SELECT COLLATION(cast(map('a', 'b') as MAP)['a'])"), - Seq(Row(sessionCollationFullyQualified))) - - checkAnswer( - sql("SELECT COLLATION(map_keys(cast(map('a', 'b') as MAP))[0])"), - Seq(Row(sessionCollationFullyQualified))) - - checkAnswer( - sql("SELECT COLLATION(cast(array('a') as ARRAY)[0])"), - Seq(Row(sessionCollationFullyQualified))) - - checkAnswer( - sql("SELECT COLLATION(cast(struct('a' as c) as STRUCT)['c'])"), - Seq(Row(sessionCollationFullyQualified))) - } - } - - test("expressions in where are aware of session collation") { - withSessionCollation("UTF8_LCASE") { - // expression in where is aware of session collation - checkAnswer(sql("SELECT 1 WHERE 'a' = 'A'"), Seq(Row(1))) - - checkAnswer(sql("SELECT 1 WHERE 'a' = cast('A' as STRING)"), Seq(Row(1))) - } - } - - test("having group by is aware of session collation") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { - sql(s"CREATE TABLE $testTable (c1 STRING) USING $dataSource") - sql(s"INSERT INTO $testTable VALUES ('a'), ('A')") - - // having clause uses session (default) collation - checkAnswer( - sql(s"SELECT COUNT(*) FROM $testTable GROUP BY c1 HAVING 'a' = 'A'"), - Seq(Row(1), Row(1))) - - // having clause uses column (implicit) collation - checkAnswer( - sql(s"SELECT COUNT(*) FROM $testTable GROUP BY c1 HAVING c1 = 'A'"), - Seq(Row(1))) - } - } - - test("min/max are aware of session collation") { - // scalastyle:off nonascii - withSessionCollationAndTable("UNICODE", testTable) { - sql(s"CREATE TABLE $testTable (c1 STRING) USING $dataSource") - sql(s"INSERT INTO $testTable VALUES ('1'), ('½')") - - checkAnswer(sql(s"SELECT MIN(c1) FROM $testTable"), Seq(Row("1"))) - - checkAnswer(sql(s"SELECT MAX(c1) FROM $testTable"), Seq(Row("½"))) - } - // scalastyle:on nonascii - } - - test("union operation with subqueries") { - withSessionCollation("UTF8_LCASE") { - checkAnswer( - sql(s""" - |SELECT 'a' = 'A' - |UNION - |SELECT 'b' = 'B' - |""".stripMargin), - Seq(Row(true))) - - checkAnswer( - sql(s""" - |SELECT 'a' = 'A' - |UNION ALL - |SELECT 'b' = 'B' - |""".stripMargin), - Seq(Row(true), Row(true))) - } - } - - test("inline table in SELECT") { - withSessionCollation("UTF8_LCASE") { - val df = s""" - |SELECT * - |FROM (VALUES ('a', 'a' = 'A')) - |""".stripMargin - - checkAnswer(sql(df), Seq(Row("a", true))) - } - } - - test("inline table in insert") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { - sql(s"CREATE TABLE $testTable (c1 STRING, c2 BOOLEAN) USING $dataSource") - - sql(s"INSERT INTO $testTable VALUES ('a', 'a' = 'A')") - checkAnswer(sql(s"SELECT * FROM $testTable"), Seq(Row("a", true))) - } - } - - test("literals in insert inherit session level collation") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { - sql(s"CREATE TABLE $testTable (c1 BOOLEAN) USING $dataSource") - - sql(s"INSERT INTO $testTable VALUES ('a' = 'A')") - sql(s"INSERT INTO $testTable VALUES (array_contains(array('a'), 'A'))") - sql(s"INSERT INTO $testTable VALUES (CONCAT(X'68656C6C6F', 'world') = 'HELLOWORLD')") - - checkAnswer(sql(s"SELECT COUNT(*) FROM $testTable WHERE c1"), Seq(Row(3))) - } - } - - // endregion } class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { test("create/alter view created from a table") { - val sessionCollation = "UTF8_LCASE" - withSessionCollationAndTable(sessionCollation, testTable) { + withTable(testTable) { sql(s"CREATE TABLE $testTable (c1 STRING, c2 STRING COLLATE UNICODE_CI) USING $dataSource") sql(s"INSERT INTO $testTable VALUES ('a', 'a'), ('A', 'A')") @@ -407,10 +272,7 @@ class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { assertTableColumnCollation(testView, "c2", "UNICODE_CI") checkAnswer( sql(s"SELECT DISTINCT COLLATION(c1), COLLATION('a') FROM $testView"), - Row(fullyQualifiedPrefix + "UTF8_BINARY", fullyQualifiedPrefix + sessionCollation)) - - // filter should use session collation - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE 'a' = 'A'"), Row(2)) + Row(fullyQualifiedPrefix + "UTF8_BINARY", fullyQualifiedPrefix + "UTF8_BINARY")) // filter should use column collation checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Row(1)) @@ -432,7 +294,7 @@ class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { assertTableColumnCollation(testView, "c2", "UNICODE_CI") checkAnswer( sql(s"SELECT DISTINCT COLLATION(c1), COLLATION('a') FROM $testView"), - Row(fullyQualifiedPrefix + "UNICODE_CI", fullyQualifiedPrefix + sessionCollation)) + Row(fullyQualifiedPrefix + "UNICODE_CI", fullyQualifiedPrefix + "UTF8_BINARY")) // after alter both rows should be returned checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Row(2)) @@ -445,7 +307,7 @@ class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { val joinTableName = "join_table" val sessionCollation = "sr" - withSessionCollationAndTable(sessionCollation, viewTableName, joinTableName) { + withTable(viewTableName, joinTableName) { sql(s"CREATE TABLE $viewTableName (c1 STRING COLLATE UNICODE_CI) USING $dataSource") sql(s"CREATE TABLE $joinTableName (c1 STRING COLLATE UTF8_LCASE) USING $dataSource") sql(s"INSERT INTO $viewTableName VALUES ('a')") @@ -473,19 +335,8 @@ class DefaultCollationTestSuiteV2 extends DefaultCollationTestSuite with Datasou override def testTable: String = s"testcat.${super.testTable}" override def testView: String = s"testcat.${super.testView}" - // delete only works on v2 - test("delete behavior") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { - sql(s"CREATE TABLE $testTable (c1 STRING) USING $dataSource") - sql(s"INSERT INTO $testTable VALUES ('a'), ('A')") - - sql(s"DELETE FROM $testTable WHERE 'a' = 'A'") - checkAnswer(sql(s"SELECT COUNT(*) FROM $testTable"), Seq(Row(0))) - } - } - test("inline table in RTAS") { - withSessionCollationAndTable("UTF8_LCASE", testTable) { + withTable(testTable) { sql(s"CREATE TABLE $testTable (c1 STRING, c2 BOOLEAN) USING $dataSource") sql(s""" |REPLACE TABLE $testTable diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index eef1f55b6a10d..b4b3bff86471e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -141,6 +141,10 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { when(t.provider).thenReturn(Some(provider)) when(t.identifier).thenReturn( ident.asTableIdentifier.copy(catalog = Some(SESSION_CATALOG_NAME))) + when(t.storage).thenReturn(CatalogStorageFormat.empty) + when(t.properties).thenReturn(Map.empty) + when(t.comment).thenReturn(None) + when(t.collation).thenReturn(None) V1Table(t) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala index 43b4a1673e2e9..a33782de8809f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala @@ -669,7 +669,7 @@ class DescribeTableSuite extends DescribeTableSuiteBase with CommandSuiteBase { QueryTest.checkAnswer( descriptionDf.filter("!(col_name in ('Created Time', 'Created By'))"), Seq( - Row("data", "string", null), + Row("data", "string collate UNICODE", null), Row("id", "bigint", null), Row("# Partition Information", "", ""), Row("# col_name", "data_type", "comment"), 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 c42599ceb7e12..0a50f07a1b2bc 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.internal -import java.util.{Locale, TimeZone} +import java.util.TimeZone import org.apache.hadoop.fs.Path import org.apache.logging.log4j.Level @@ -508,31 +508,6 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { |""".stripMargin) } - test("SPARK-47765: set collation") { - Seq("UNICODE", "UNICODE_CI", "utf8_lcase", "utf8_binary").foreach { collation => - sql(s"set collation $collation") - assert(sqlConf.getConf(SQLConf.DEFAULT_COLLATION) === collation.toUpperCase(Locale.ROOT)) - } - - checkError( - exception = intercept[SparkIllegalArgumentException] { - sql(s"SET COLLATION unicode_c").collect() - }, - condition = "INVALID_CONF_VALUE.DEFAULT_COLLATION", - parameters = Map( - "confValue" -> "UNICODE_C", - "confName" -> "spark.sql.session.collation.default", - "proposals" -> "UNICODE" - )) - - withSQLConf(SQLConf.TRIM_COLLATION_ENABLED.key -> "false") { - checkError( - exception = intercept[AnalysisException](sql(s"SET COLLATION UNICODE_CI_RTRIM")), - condition = "UNSUPPORTED_FEATURE.TRIM_COLLATION" - ) - } - } - test("SPARK-43028: config not found error") { checkError( exception = intercept[SparkNoSuchElementException](spark.conf.get("some.conf")),