You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "mihailom-db (via GitHub)" <gi...@apache.org> on 2024/03/05 08:04:09 UTC

[PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

mihailom-db opened a new pull request, #45383:
URL: https://github.com/apache/spark/pull/45383

   ### What changes were proposed in this pull request?
   This PR adds automatic casting and collations resolution as per `PGSQL` behaviour:
   
   1. Collations set on the metadata level are implicit
   2. Collations set using the `COLLATE` expression are explicit
   3. When there is a combination of expressions of multiple collations the output will be:
   - if there are explicit collations and all of them are equal then that collation will be the output
   - if there are multiple different explicit collations `COLLATION_MISMATCH.EXPLICIT` will be thrown
   - if there are no explicit collations and only a single type of non default collation, that one will be used
   - if there are no explicit collations and multiple non-default implicit ones `COLLATION_MISMATCH.IMPLICIT` will be thrown
   
   
   Another thing is that `INDETERMINATE_COLLATION` should only be thrown on comparison operations, and we should be able to combine different implicit collations for certain operations like concat and possible others in the future.
   This is why I had to add another predefined collation id named INDETERMINATE_COLLATION_ID which means that the result is a combination of conflicting non-default implicit collations. Right now it has an id of -1 so it fails if it ever goes to the `CollatorFactory`.
   
   
   ### Why are the changes needed?
   We need to be able to compare columns and values with different collations and set a way of explicitly changing the collation we want to use.
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. We add 3 new errors and enable collation casting.
   
   
   ### How was this patch tested?
   Tests in `CollationSuite` were done to check code validity.
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1528942886


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +770,177 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat

Review Comment:
   Why are you special casing `Concat` expression here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #45383:
URL: https://github.com/apache/spark/pull/45383#issuecomment-1991172536

   Some high-level questions:
   1. If a function requires certain collations but the input uses a different collation, shall we implicitly cast or fail?
   2. If a function's inputs do not use the same collation, shall we implicit cast or fail?
   3. If we cast a string with collation to integer or datetime, do we need to consider the collation?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1544004871


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -467,6 +467,24 @@
     ],
     "sqlState" : "42704"
   },
+  "COLLATION_MISMATCH" : {
+    "message" : [
+      "Could not determine which collation to use for string comparison."

Review Comment:
   shall we say "string functions and operators"?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546065154


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression

Review Comment:
   This is a good point that we can't reuse the existing `fold` style to do implicit cast for string collations. If we must rewrite the code, a new rule sounds better.
   
   To avoid the notorious analyzer rule execution order issues, let's make sure this new rule is orthogonal to existing implicit cast rules: the existing "fold" style implicit cast should not apply to string type with different collations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547843264


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)

Review Comment:
   There is one problem with this. We said collations are only different if their collationId's are different. If we do this, we would need to change the line in else clause to something that does not look so nice. I have changed this structure in https://github.com/apache/spark/pull/45819, which is a followup for this PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1549240904


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType}
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, CaseWhen, Cast, Coalesce, Collate, Concat, ConcatWs, CreateArray, Expression, Greatest, If, In, InSubquery, Least, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case ifExpr: If =>
+      ifExpr.withNewChildren(
+        ifExpr.predicate +: collateToSingleType(Seq(ifExpr.trueValue, ifExpr.falseValue)))
+    case caseWhenExpr: CaseWhen =>
+      val newValues = collateToSingleType(
+        caseWhenExpr.branches.map(b => b._2) ++ caseWhenExpr.elseValue)
+      caseWhenExpr.withNewChildren(
+        interleave(Seq.empty, caseWhenExpr.branches.map(b => b._1), newValues))
+    case substrExpr: Substring =>
+      // This case is necessary for changing Substring input to implicit collation
+      substrExpr.withNewChildren(
+        collateToSingleType(Seq(substrExpr.str)) :+ substrExpr.pos :+ substrExpr.len)

Review Comment:
   For now we do not need it, I can revert this change and leave it for default collation handling. Because if we add a flag for collation priority, we will need to change the priority of the input to implicit, if it was default before.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547193589


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)

Review Comment:
   How do we define implicit collation? The collation can come from multiple places:
   1. default collation
   2. table column collation
   3. was explicit collation before but does not get propagate
   
   Why do we special case default collation?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547189187


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)

Review Comment:
   This is the tricky part of adding a new rule. We must make sure we follow the behavior of existing implicit cast rules and only add implicit cast for certain children of an expression. For example, the true and false branches of `If` expression need implicit cast, but not the if condition. `ExpectsInputTypes` does not indicate that the expression requires all its children to be the same type, and should not be handled here.
   
   Please carefully check all implicit cast rules and revisit this new rule. @mihailom-db 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547259400


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>

Review Comment:
   Then please match the ConcatWs expression explicitly to handle this case. What I disagree with is to do this blindly for all expressions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546285554


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   why only `CreateArray` can preserve explicit collation?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519581505


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {

Review Comment:
   I would like to see tests for this. Can you add them to `TypeCoercionSuite`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on PR #45383:
URL: https://github.com/apache/spark/pull/45383#issuecomment-1994569802

   > Some high-level questions:
   > 
   > 1. If a function requires certain collations but the input uses a different collation, shall we implicitly cast or fail?
   > 2. If a function's inputs do not use the same collation, shall we implicit cast or fail?
   > 3. If we cast a string with collation to integer or datetime, do we need to consider the collation?
   
   @cloud-fan - these are great questions and I think that they should be part of the spec.
   Rough answers from my side are:
   1) I think that we should fail but I think that there are some subtle caveats here that should be covered in the design spec.
   2) Depends on a function. e.g. for contains we should fail. For concat we should succeed.
   3) No. Decimal/datetime formatting should be part of "language settings" which are not part of collation track.
   
   @mihailom-db will extend casting section of the doc. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530323337


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,184 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat
+        if shouldCast(checkCastWithIndeterminate.children) =>
+        val newChildren =
+          collateToSingleType(checkCastWithIndeterminate.children, failOnIndeterminate = false)
+        checkCastWithIndeterminate.withNewChildren(newChildren)
+
+      case checkCastWithoutIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if shouldCast(checkCastWithoutIndeterminate.children) =>
+        val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+        checkCastWithoutIndeterminate.withNewChildren(newChildren)
+
+      case checkIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if hasIndeterminate(checkIndeterminate.children
+          .filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))) =>
+        throw QueryCompilationErrors.indeterminateCollationError()
+
+      case checkImplicitCastInputTypes: ImplicitCastInputTypes
+        if checkImplicitCastInputTypes.children.exists(e => hasStringType(e.dataType))
+          && checkImplicitCastInputTypes.inputTypes.nonEmpty =>
+        val collationId: Int =
+          getOutputCollation(checkImplicitCastInputTypes
+            .children.filter { e => hasStringType(e.dataType) })
+        val children: Seq[Expression] = checkImplicitCastInputTypes
+          .children.zip(checkImplicitCastInputTypes.inputTypes).map {
+            case (e, st) if hasStringType(st) =>
+              castStringType(e, collationId, Some(st)).getOrElse(e)
+            case (e, TypeCollection(types)) if types.exists(hasStringType) =>
+              types.flatMap{ dt =>
+                if (hasStringType(dt)) {
+                  castStringType(e, collationId, Some(dt))
+                } else {
+                  implicitCast(e, dt)
+                }
+              }.headOption.getOrElse(e)
+            case (in, expected) => implicitCast(in, expected).getOrElse(in)
+          }
+        checkImplicitCastInputTypes.withNewChildren(children)
+
+      case checkExpectsInputType: ExpectsInputTypes

Review Comment:
   Also, I was a bit sceptical about changing acceptsType in StringType. If I understood correctly this function should give us what types can be used for this expression without implicitCasting. Unfortunately, even after Uros'es PR https://github.com/apache/spark/pull/45422 we will have this problem. We do not want to get a function with two binary collations like UNICODE and UTF8_BINARY, and still say that it is fine to call a function. We specifically need one collation. One workaround would be to just include collationid in the check of all the special cases. This also would change the hierarchy of classes, as additional objects that Uros added would need to extend StringType.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530283087


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +770,177 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat

Review Comment:
   AFAIK it is part of the spec and PGSQL has the same behavior - @srielau as FYI.
   
   Idea is that you can have indeterminate collations as the result of some expressions (not sure what else is there outside of concat?). Such strings can't be pushed to storage and most expressions don't accept it as an input. But you can do concat across different collations and then cast final output to the final collation.
   
   That being said - all of this is far from obvious and I think that you at least need some comments in the code that explain this. I would also prefer new trait for this, even though `concat` is the only user.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513955577


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will be.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 => throw QueryCompilationErrors.explicitCollationMismatchError(
+          explicitTypes.head.simpleString, explicitTypes.tail.head.simpleString)
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (isIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.indeterminateCollationError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else if (hasMultipleImplicits(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.implicitCollationMismatchError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else {
+            dataTypes.find(!_.isDefaultCollation)
+              .getOrElse(StringType)
+              .collationId
+          }
+      }
+    }
+
+    private def isIndeterminate(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.exists(_.isIndeterminateCollation)
+
+
+    private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.filter(!_.isDefaultCollation).distinct.size > 1
+
+    private def hasExplicitCollation(expression: Expression): Boolean = {
+      if (!expression.dataType.isInstanceOf[StringType]) {
+        false
+      }
+      else {
+        expression match {
+          case _: Collate => true

Review Comment:
   ```suggestion
             case Collate(_, _) => true
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513956640


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will be.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 => throw QueryCompilationErrors.explicitCollationMismatchError(
+          explicitTypes.head.simpleString, explicitTypes.tail.head.simpleString)
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (isIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.indeterminateCollationError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else if (hasMultipleImplicits(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.implicitCollationMismatchError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else {
+            dataTypes.find(!_.isDefaultCollation)
+              .getOrElse(StringType)
+              .collationId
+          }
+      }
+    }
+
+    private def isIndeterminate(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.exists(_.isIndeterminateCollation)
+
+
+    private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.filter(!_.isDefaultCollation).distinct.size > 1
+
+    private def hasExplicitCollation(expression: Expression): Boolean = {
+      if (!expression.dataType.isInstanceOf[StringType]) {
+        false
+      }
+      else {
+        expression match {
+          case _: Collate => true

Review Comment:
   This is the only possible change for here. As `Collate` constructor expects 2 parameters and scala does not want to match only on `Collate`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513042604


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will be.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 => throw QueryCompilationErrors.explicitCollationMismatchError(
+          explicitTypes.head.simpleString, explicitTypes.tail.head.simpleString)

Review Comment:
   can we maybe make this method and error accept a sequence of collations and not just the first two?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513032494


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -958,14 +1062,16 @@ object TypeCoercion extends TypeCoercionBase {
 
   override def implicitCast(e: Expression, expectedType: AbstractDataType): Option[Expression] = {
     implicitCast(e.dataType, expectedType).map { dt =>
-      if (dt == e.dataType) e else Cast(e, dt)
+      if (dt == e.dataType) { e }

Review Comment:
   if the whole if statement isn't in one line statements shouldn't be on the same line as the condition
   see https://github.com/databricks/scala-style-guide?tab=readme-ov-file#curly-braces



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519762760


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -138,21 +140,31 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   @scala.annotation.tailrec
   private def findWiderTypeForString(dt1: DataType, dt2: DataType): Option[DataType] = {
     (dt1, dt2) match {
-      case (StringType, _: IntegralType) => Some(LongType)
-      case (StringType, _: FractionalType) => Some(DoubleType)
-      case (StringType, NullType) => Some(StringType)
+      case (_: StringType, _: IntegralType) => Some(LongType)
+      case (_: StringType, _: FractionalType) => Some(DoubleType)
+      case (st: StringType, NullType) => Some(st)
       // If a binary operation contains interval type and string, we can't decide which
       // interval type the string should be promoted as. There are many possible interval
       // types, such as year interval, month interval, day interval, hour interval, etc.
-      case (StringType, _: AnsiIntervalType) => None
-      case (StringType, a: AtomicType) => Some(a)
-      case (other, StringType) if other != StringType => findWiderTypeForString(StringType, other)
+      case (_: StringType, _: AnsiIntervalType) => None
+      case (_: StringType, a: AtomicType) => Some(a)
+      case (other, st: StringType) if !other.isInstanceOf[StringType] =>
+        findWiderTypeForString(st, other)
       case _ => None
     }
   }
 
-  override def findWiderCommonType(types: Seq[DataType]): Option[DataType] = {
-    types.foldLeft[Option[DataType]](Some(NullType))((r, c) =>
+  override def findWiderCommonType(exprs: Seq[Expression],
+                                   failOnIndeterminate: Boolean = false): Option[DataType] = {
+    (if (exprs.map(_.dataType).partition(hasStringType)._1.distinct.size > 1) {
+      val collationId = CollationTypeCasts.getOutputCollation(exprs, failOnIndeterminate)
+      exprs.map(e =>
+        if (hasStringType(e.dataType)) {
+          castStringType(e.dataType, collationId)
+          e
+        }
+        else e)
+    } else exprs).map(_.dataType).foldLeft[Option[DataType]](Some(NullType))((r, c) =>
       r match {
         case Some(d) => findWiderTypeForTwo(d, c)

Review Comment:
   `findWiderTypeForTwo` only accepts `dataType`'s. To extract if we have explicit collation, we need the expression itself. I will look into `findWiderTypeForTwo` and whether it makes sense to change it to accept expressions as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519567374


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -138,21 +140,31 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   @scala.annotation.tailrec
   private def findWiderTypeForString(dt1: DataType, dt2: DataType): Option[DataType] = {
     (dt1, dt2) match {
-      case (StringType, _: IntegralType) => Some(LongType)
-      case (StringType, _: FractionalType) => Some(DoubleType)
-      case (StringType, NullType) => Some(StringType)
+      case (_: StringType, _: IntegralType) => Some(LongType)
+      case (_: StringType, _: FractionalType) => Some(DoubleType)
+      case (st: StringType, NullType) => Some(st)
       // If a binary operation contains interval type and string, we can't decide which
       // interval type the string should be promoted as. There are many possible interval
       // types, such as year interval, month interval, day interval, hour interval, etc.
-      case (StringType, _: AnsiIntervalType) => None
-      case (StringType, a: AtomicType) => Some(a)
-      case (other, StringType) if other != StringType => findWiderTypeForString(StringType, other)
+      case (_: StringType, _: AnsiIntervalType) => None
+      case (_: StringType, a: AtomicType) => Some(a)
+      case (other, st: StringType) if !other.isInstanceOf[StringType] =>
+        findWiderTypeForString(st, other)
       case _ => None
     }
   }
 
-  override def findWiderCommonType(types: Seq[DataType]): Option[DataType] = {
-    types.foldLeft[Option[DataType]](Some(NullType))((r, c) =>
+  override def findWiderCommonType(exprs: Seq[Expression],
+                                   failOnIndeterminate: Boolean = false): Option[DataType] = {
+    (if (exprs.map(_.dataType).partition(hasStringType)._1.distinct.size > 1) {

Review Comment:
   If I am reading this correctly you are just trying to figure out whether there is a expr with `StringType`? Can't you just say `exprs.exists(_.dataType.HasStringType)`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519561653


##########
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##########
@@ -41,14 +41,22 @@ class StringType private(val collationId: Int) extends AtomicType with Serializa
    */
   def isBinaryCollation: Boolean = CollationFactory.fetchCollation(collationId).isBinaryCollation
 
+  /**
+   * Returns whether the collation is indeterminate. An indeterminate collation is
+   * a result of combination of conflicting non-default implicit collations.
+   */
+  def isIndeterminateCollation: Boolean = collationId == CollationFactory.INDETERMINATE_COLLATION_ID
+
   /**
    * Type name that is shown to the customer.
    * If this is an UCS_BASIC collation output is `string` due to backwards compatibility.
    */
   override def typeName: String =
     if (isDefaultCollation) "string"
+    else if (isIndeterminateCollation) s"string COLLATE INDETERMINATE_COLLATION"
     else s"string COLLATE ${CollationFactory.fetchCollation(collationId).collationName}"

Review Comment:
   This change will require changes on PySpark side.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530217520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -64,7 +65,7 @@ abstract class TypeCoercionBase {
    * is larger than decimal, and yet decimal is more precise than double, but in
    * union we would cast the decimal into double.
    */
-  def findWiderCommonType(types: Seq[DataType]): Option[DataType]
+  def findWiderCommonType(children: Seq[DataType]): Option[DataType]

Review Comment:
   why do we rename the parameter?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on PR #45383:
URL: https://github.com/apache/spark/pull/45383#issuecomment-2017420288

   Should I add an additional check if we call fetchCollation of -1? Since implicit casting takes all expressions into account, it might happen that some were missed and we would risk getting indexOutOfBoundError. For currently implemented functions for collated strings all should work.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1544061247


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    // Case when we do fail if resulting collation is indeterminate
+    case checkCastWithoutIndeterminate @ (_: BinaryExpression
+                                          | _: Predicate
+                                          | _: SortOrder
+                                          | _: ExpectsInputTypes
+                                          | _: ComplexTypeMergingExpression
+                                          | _: CreateArray)
+      if shouldCast(checkCastWithoutIndeterminate.children) =>
+      val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+      checkCastWithoutIndeterminate.withNewChildren(newChildren)
+  }
+
+  def shouldCast(types: Seq[Expression]): Boolean = {
+    types.filter(e => hasStringType(e.dataType))
+      .map(e => extractStringType(e.dataType).collationId).distinct.size > 1
+  }
+
+  /**
+   * Checks whether given data type contains StringType.
+   */
+  @tailrec
+  def hasStringType(dt: DataType): Boolean = dt match {
+    case _: StringType => true
+    case ArrayType(et, _) => hasStringType(et)
+    case _ => false
+  }
+
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, collationId: Int): Option[Expression] =
+    castStringType(expr.dataType, collationId).map { dt =>
+      if (dt == expr.dataType) expr else Cast(expr, dt)
+    }
+
+  private def castStringType(inType: AbstractDataType, collationId: Int): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId == collationId => st
+      case _: StringType => StringType(collationId)
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, collationId).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val collationId = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, collationId).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(exprs: Seq[Expression]): Int = {
+    val explicitTypes = exprs.filter(hasExplicitCollation)
+      .map(e => extractStringType(e.dataType).collationId).distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => explicitTypes.head
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val dataTypes = exprs.filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))
+
+        if (hasMultipleImplicits(dataTypes)) {
+          throw QueryCompilationErrors.implicitCollationMismatchError()
+        }
+        else {
+          dataTypes.find(dt => !(dt == SQLConf.get.defaultStringType))
+            .getOrElse(SQLConf.get.defaultStringType)
+            .collationId
+        }
+    }
+  }
+
+  /**
+   * This check is always preformed when we have no explicit collation. It returns true
+   * if there are more than one implicit collations. Collations are distinguished by their
+   * collationId.
+   * @param dataTypes
+   * @return
+   */
+  private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+    dataTypes.filter(dt => !(dt == SQLConf.get.defaultStringType))
+      .map(_.collationId).distinct.size > 1
+
+  /**
+   * Checks if a given expression has explicitly set collation. For complex DataTypes
+   * we need to check nested children.
+   * @param expression
+   * @return
+   */
+  private def hasExplicitCollation(expression: Expression): Boolean = {
+    expression match {
+      case _: Collate => true
+      case e if e.dataType.isInstanceOf[ArrayType]
+      => expression.children.exists(hasExplicitCollation)

Review Comment:
   This looks a bit hacky. What is the exact definition of explicit collation?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1544741676


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression

Review Comment:
   @cloud-fan Is this what you meant? Now this type of query will be broken `SELECT array('b' collate unicode_ci) || 'a' collate unicode`, but I can add support for array explicit as a followup.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547226617


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,26 +707,39 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)
+        }
+        val st = getOutputCollation(e.children)
+        val children: Seq[Expression] = childrenBeforeCollations.map {

Review Comment:
   Well the problem is that we have TypeCollections as expected types. If someone defines StringType as non-first DataType (and consequently non-defaultConcreteType), we need to go through previous elements first to see if it will come to StringType cast. If it does, we need to cast it to the same collation. For example, look at `ConcatWs` if it receives a BinaryType, it should try first to cast to ArrayType, get None and then try to cast it to StringType. But on the other hand, we need for consistency with findWiderCommonType to cast collations at the beginning of rules, as we would get a breakage in castings otherwise.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547201438


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)

Review Comment:
   This rule is only touching children that have StringType's as their DataType. I can reorder execution to first filter out StringType arguments and then work on them, Agreed for `ExpectInputTypes`, I have checked now and it should not have any function with 2 StringType inputs, so we are safe not to cast it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547197835


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,26 +707,39 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)
+        }
+        val st = getOutputCollation(e.children)
+        val children: Seq[Expression] = childrenBeforeCollations.map {

Review Comment:
   what are we doing here? `ImplicitCastInputTypes` defines the expected data type of each input, but does not require all inputs to be the same data type.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,26 +707,39 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)
+        }
+        val st = getOutputCollation(e.children)
+        val children: Seq[Expression] = childrenBeforeCollations.map {
+          case in if hasStringType(in.dataType) =>
+            castStringType(in, st).getOrElse(in)
+          case in => in
         }
         e.withNewChildren(children)
 
       case e: ExpectsInputTypes if e.inputTypes.nonEmpty =>
         // Convert NullType into some specific target type for ExpectsInputTypes that don't do
         // general implicit casting.
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] =

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519730371


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -138,21 +140,31 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   @scala.annotation.tailrec
   private def findWiderTypeForString(dt1: DataType, dt2: DataType): Option[DataType] = {
     (dt1, dt2) match {
-      case (StringType, _: IntegralType) => Some(LongType)
-      case (StringType, _: FractionalType) => Some(DoubleType)
-      case (StringType, NullType) => Some(StringType)
+      case (_: StringType, _: IntegralType) => Some(LongType)
+      case (_: StringType, _: FractionalType) => Some(DoubleType)
+      case (st: StringType, NullType) => Some(st)
       // If a binary operation contains interval type and string, we can't decide which
       // interval type the string should be promoted as. There are many possible interval
       // types, such as year interval, month interval, day interval, hour interval, etc.
-      case (StringType, _: AnsiIntervalType) => None
-      case (StringType, a: AtomicType) => Some(a)
-      case (other, StringType) if other != StringType => findWiderTypeForString(StringType, other)
+      case (_: StringType, _: AnsiIntervalType) => None
+      case (_: StringType, a: AtomicType) => Some(a)
+      case (other, st: StringType) if !other.isInstanceOf[StringType] =>
+        findWiderTypeForString(st, other)
       case _ => None
     }
   }
 
-  override def findWiderCommonType(types: Seq[DataType]): Option[DataType] = {
-    types.foldLeft[Option[DataType]](Some(NullType))((r, c) =>
+  override def findWiderCommonType(exprs: Seq[Expression],
+                                   failOnIndeterminate: Boolean = false): Option[DataType] = {
+    (if (exprs.map(_.dataType).partition(hasStringType)._1.distinct.size > 1) {

Review Comment:
   I see. Still, I think partition + select first in tuple is pretty weird way to filter?
   Shouldn't `exprs.map(_.dataType).filter(hasStringType).distinct.size ` be more performant and readable?
   
   All of that + comment that you shouldn't be using `distinct` against `StringType` for checking collation equivalence since `StringType` may in future hold other properties. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519752158


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expressions to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will have.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 =>
+          throw QueryCompilationErrors
+            .explicitCollationMismatchError(
+              explicitTypes.map(t => t.asInstanceOf[StringType].typeName)
+            )
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (hasIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.indeterminateCollationError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else if (hasMultipleImplicits(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.implicitCollationMismatchError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else {
+            dataTypes.find(!_.isDefaultCollation)
+              .getOrElse(StringType)
+              .collationId

Review Comment:
   That is right, if we get to this point, we have only one implicit collation. In case we have 0 implicit non-default collations, we get to default collation and we return id.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519323364


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expressions to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will have.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {

Review Comment:
   i thought about this for a bit and we should probably improve this logic so that the explicit collation has the utmost precedence, ie if there is a one input with explicit and another with indeterminate the output should be explicit as well
   
   the precedence hierarchy should probably go like this:
   1. no collation
   2. implicit collation
   3. indeterminate collation
   4. explicit collation
   
   what do you think about this approach? It seems to me that this is the pgsql behaviour as well



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519324890


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expressions to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will have.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 =>
+          throw QueryCompilationErrors
+            .explicitCollationMismatchError(
+              explicitTypes.map(t => t.asInstanceOf[StringType].typeName)
+            )
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (hasIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {

Review Comment:
   also a sidenote about failOnIndeterminate; currently if will result sometimes in analysis and sometimes in runtime. Is this okay and should we maybe only fail in one place?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530424022


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion
+      .CollationTypeCasts
+      .getOutputCollation(Seq(left, right))

Review Comment:
   Yes, getOutputCollation throws exception. This PR has to go in after https://github.com/apache/spark/pull/45422. Will resolve conflicts and deal with exception throwing when that happens.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530311503


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +770,177 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat

Review Comment:
   ok, so it kind of delays the determining of the collation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530225397


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,184 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {

Review Comment:
   Can we put it in a new file?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1544588809


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression

Review Comment:
   This looks like a clean solution for string collation implicit cast, but actually it is not. The current design of the Spark implicit cast system is: we define some core functions like `findTightestCommonType`, `findWiderTypeForTwo`, etc., and there are a bunch of rules to find the target expressions to apply implicit cast using these core functions.
   
   The string collation implicit cast should be part of the core functions, instead of a new rule. Otherwise this new rule is a bit inconsistent with the current system and we need to replicate the matching of expressions that need implicit cast.
   
   Since the core functions only take `DataType`, not `Expression`, I suggest we add a new boolean flag to `StringType` to indicate it's explicit or not.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546284265


##########
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##########
@@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.util.CollationFactory
  */
 @Stable
 class StringType private(val collationId: Int) extends AtomicType with Serializable {
+  var isExplicit: Boolean = false

Review Comment:
   why isn't it part of construct parameters?



##########
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##########
@@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.util.CollationFactory
  */
 @Stable
 class StringType private(val collationId: Int) extends AtomicType with Serializable {
+  var isExplicit: Boolean = false

Review Comment:
   why isn't it part of constructor parameters?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546396114


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>

Review Comment:
   Added Elt to the check, but will probably revisit that function when we enter a PR for it's support.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547239943


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,26 +707,39 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)
+        }
+        val st = getOutputCollation(e.children)
+        val children: Seq[Expression] = childrenBeforeCollations.map {

Review Comment:
   It seems we are fixing the problem at the wrong place. If `TypeCollections` has issues, let's update the code in `def implicitCast` to fix it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1549240911


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala:
##########
@@ -112,12 +112,14 @@ case class Collate(child: Expression, collationName: String)
   since = "4.0.0",
   group = "string_funcs")
 // scalastyle:on line.contains.tab
-case class Collation(child: Expression) extends UnaryExpression with RuntimeReplaceable {
-  override def dataType: DataType = StringType
+case class Collation(child: Expression)
+  extends UnaryExpression with RuntimeReplaceable with ExpectsInputTypes {
+  override def dataType: DataType = SQLConf.get.defaultStringType

Review Comment:
   We shouldn't override it. The default implementation is better `replacement.dataType`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1548791139


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType}
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, CaseWhen, Cast, Coalesce, Collate, Concat, ConcatWs, CreateArray, Expression, Greatest, If, In, InSubquery, Least, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc@(_: In
+             | _: InSubquery
+             | _: CreateArray
+             | _: If
+             | _: ArrayJoin
+             | _: CaseWhen
+             | _: Concat
+             | _: Greatest
+             | _: Least
+             | _: Coalesce
+             | _: BinaryExpression
+             | _: ConcatWs
+             | _: Substring) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)
+          .distinctBy(_.collationId)
+
+        if (hasMultipleImplicits(implicitTypes)) {

Review Comment:
   this can be `implicitTypes.length > 1` now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547242430


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>

Review Comment:
   A simple example is ConcatWs. It can have ArrayType(StringType, _) for input strings and StringType for separator as parameters. What collations do we want for this then? We need to cast the ArrayType into a proper collation if separator is explicit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513034188


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will be.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 => throw QueryCompilationErrors.explicitCollationMismatchError(
+          explicitTypes.head.simpleString, explicitTypes.tail.head.simpleString)
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (isIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.indeterminateCollationError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else if (hasMultipleImplicits(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.implicitCollationMismatchError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else {
+            dataTypes.find(!_.isDefaultCollation)
+              .getOrElse(StringType)
+              .collationId
+          }
+      }
+    }
+
+    private def isIndeterminate(dataTypes: Seq[StringType]): Boolean =

Review Comment:
   maybe `hasIndeterminate` would make more sense here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513038768


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will be.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 => throw QueryCompilationErrors.explicitCollationMismatchError(
+          explicitTypes.head.simpleString, explicitTypes.tail.head.simpleString)
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (isIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {

Review Comment:
   this can be a private method



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513053988


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will be.

Review Comment:
   ```suggestion
        * a collation type which the output will have.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513053311


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.

Review Comment:
   ```suggestion
        *  Collates the input expressions to a single collation.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513061821


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion

Review Comment:
   why not add `StringPredicate` to the rule check like for `BinaryComparison`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion

Review Comment:
   why not just add `StringPredicate` to the rule check like for `BinaryComparison`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546551263


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   The example query is `SELECT array('a', 'b' collate UNICODE) || ('c' collate UNICODE_CI)`. This is kind of artificial as users can write one `array(...)` function, but think about a rewrite like `SELECT arr || ('c' collate UNICODE_CI) FROM (SELECT array('a', 'b' collate UNICODE) as arr)`, I think returning an array with string of collation UNICODE_CI is fine?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547226867


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,26 +707,39 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)
+        }
+        val st = getOutputCollation(e.children)
+        val children: Seq[Expression] = childrenBeforeCollations.map {
+          case in if hasStringType(in.dataType) =>
+            castStringType(in, st).getOrElse(in)
+          case in => in
         }
         e.withNewChildren(children)
 
       case e: ExpectsInputTypes if e.inputTypes.nonEmpty =>
         // Convert NullType into some specific target type for ExpectsInputTypes that don't do
         // general implicit casting.
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] =

Review Comment:
   This is not necessary, agreed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546046606


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression

Review Comment:
   Also, I do not believe incorporating string collations into core functions is the best way to go. If you take a look at findWiderCommonForTwo, it is implemented differently for AnsiTypeCoercion and TypeCoercion. Firstly both of those take leftFold and go through expressions in order, but in our case that is not what we want. We want to first know what collations are explicit and then look for anything else. This is important because we want to have clear design on which error has to come first, and explicit mismatch should always come before implicit one. Another thing is that AnsiTypeCoercion seems to behave differently for StringTypes, we do not reorder any type in the sequence, which would result in implementing the same thing we did here with addition of another rule, but just in a core, already well defined function.
   One more thing I would add is that not all rules actually use core functions, e.g. ConcatCoercion. Also doing collations casting in one rule is way more efficient, as otherwise we would have to reorder StringTypes in every other rule, to make sure explicit collation mismatches are thrown first, which would result in multiple reorderings as opposed to constant of 2, one at the beginning for all StringTypes and the other at the end for all expressions that were cast to StringType from other types in some other implicit casting rules.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547845857


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)

Review Comment:
   I could change the else branch only here as well, to make it look nicer as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1548790383


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType}
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, CaseWhen, Cast, Coalesce, Collate, Concat, ConcatWs, CreateArray, Expression, Greatest, If, In, InSubquery, Least, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc@(_: In
+             | _: InSubquery
+             | _: CreateArray
+             | _: If

Review Comment:
   We should add more case matches as some expressions do not require all its inputs to be the same type
   ```
   case if: If =>
     if.withNewChildren(if.predicate +: collateToSingleType(if.trueValue, if.falseValue))
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547812585


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)
+
+        if (hasMultipleImplicits(implicitTypes)) {
+          throw QueryCompilationErrors.implicitCollationMismatchError()
+        }
+        else {
+          implicitTypes.find(dt => !(dt == SQLConf.get.defaultStringType))
+            .getOrElse(SQLConf.get.defaultStringType)
+        }
+    }
+  }
+
+  /**
+   * This check is always preformed when we have no explicit collation. It returns true
+   * if there are more than one implicit collations. Collations are distinguished by their
+   * collationId.
+   * @param dataTypes
+   * @return
+   */
+  private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+    dataTypes.map(_.collationId)
+      .filter(dt => !(dt == SQLConf.get.defaultStringType.collationId)).distinct.size > 1
+
+}
+
+/**
+ * This rule is used to collate all existing expressions related to StringType into a single
+ * collation. Arrays are handled using their elementType and should be cast for these expressions.
+ */
+object PreCollationTypeCasts extends CollationTypeCasts {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc@(_: In
+             | _: InSubquery
+             | _: CreateArray
+             | _: ComplexTypeMergingExpression
+             | _: ArrayJoin
+             | _: BinaryExpression
+             | _: ConcatWs
+             | _: Substring) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+}
+
+/**
+ *  This rule is used for managing expressions that have possible implicit casts from different
+ *  types in ImplicitTypeCasts rule.
+ */
+object PostCollationTypeCasts extends CollationTypeCasts {

Review Comment:
   why does this rule need two phases?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519579806


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1

Review Comment:
   Let's again be explicit here and check whether collation ids are different.
   In future we may want to extend `StringType` with other fields (e.g. max lenght) and for such strings this rule shouldn't kick in.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519574637


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -138,21 +140,31 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   @scala.annotation.tailrec
   private def findWiderTypeForString(dt1: DataType, dt2: DataType): Option[DataType] = {
     (dt1, dt2) match {
-      case (StringType, _: IntegralType) => Some(LongType)
-      case (StringType, _: FractionalType) => Some(DoubleType)
-      case (StringType, NullType) => Some(StringType)
+      case (_: StringType, _: IntegralType) => Some(LongType)
+      case (_: StringType, _: FractionalType) => Some(DoubleType)
+      case (st: StringType, NullType) => Some(st)
       // If a binary operation contains interval type and string, we can't decide which
       // interval type the string should be promoted as. There are many possible interval
       // types, such as year interval, month interval, day interval, hour interval, etc.
-      case (StringType, _: AnsiIntervalType) => None
-      case (StringType, a: AtomicType) => Some(a)
-      case (other, StringType) if other != StringType => findWiderTypeForString(StringType, other)
+      case (_: StringType, _: AnsiIntervalType) => None
+      case (_: StringType, a: AtomicType) => Some(a)
+      case (other, st: StringType) if !other.isInstanceOf[StringType] =>
+        findWiderTypeForString(st, other)
       case _ => None
     }
   }
 
-  override def findWiderCommonType(types: Seq[DataType]): Option[DataType] = {
-    types.foldLeft[Option[DataType]](Some(NullType))((r, c) =>
+  override def findWiderCommonType(exprs: Seq[Expression],
+                                   failOnIndeterminate: Boolean = false): Option[DataType] = {
+    (if (exprs.map(_.dataType).partition(hasStringType)._1.distinct.size > 1) {
+      val collationId = CollationTypeCasts.getOutputCollation(exprs, failOnIndeterminate)
+      exprs.map(e =>
+        if (hasStringType(e.dataType)) {
+          castStringType(e.dataType, collationId)
+          e
+        }
+        else e)
+    } else exprs).map(_.dataType).foldLeft[Option[DataType]](Some(NullType))((r, c) =>
       r match {
         case Some(d) => findWiderTypeForTwo(d, c)

Review Comment:
   I find this pretty weird.
   Why can't we just rely on `fold` + `findWiderTypeForTwo` logic? I think that type checks should remain `foldable` even with collation concept? i.e. we should always be able to determine output collation by just comparing two expressions and eventually folding to the result?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "uros-db (via GitHub)" <gi...@apache.org>.
uros-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1515613758


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion

Review Comment:
   also note that in:
   `final lazy val collationId: Int = left.dataType.asInstanceOf[StringType].collationId`
   
   `left` is chosen arbitrarily, and shouldn't be used to check whether a function supports that collation type _before_ checking whether `right` has the same collation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530316018


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,184 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat
+        if shouldCast(checkCastWithIndeterminate.children) =>
+        val newChildren =
+          collateToSingleType(checkCastWithIndeterminate.children, failOnIndeterminate = false)
+        checkCastWithIndeterminate.withNewChildren(newChildren)
+
+      case checkCastWithoutIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if shouldCast(checkCastWithoutIndeterminate.children) =>
+        val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+        checkCastWithoutIndeterminate.withNewChildren(newChildren)
+
+      case checkIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if hasIndeterminate(checkIndeterminate.children
+          .filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))) =>
+        throw QueryCompilationErrors.indeterminateCollationError()
+
+      case checkImplicitCastInputTypes: ImplicitCastInputTypes
+        if checkImplicitCastInputTypes.children.exists(e => hasStringType(e.dataType))
+          && checkImplicitCastInputTypes.inputTypes.nonEmpty =>
+        val collationId: Int =
+          getOutputCollation(checkImplicitCastInputTypes
+            .children.filter { e => hasStringType(e.dataType) })
+        val children: Seq[Expression] = checkImplicitCastInputTypes
+          .children.zip(checkImplicitCastInputTypes.inputTypes).map {
+            case (e, st) if hasStringType(st) =>
+              castStringType(e, collationId, Some(st)).getOrElse(e)
+            case (e, TypeCollection(types)) if types.exists(hasStringType) =>
+              types.flatMap{ dt =>
+                if (hasStringType(dt)) {
+                  castStringType(e, collationId, Some(dt))
+                } else {
+                  implicitCast(e, dt)
+                }
+              }.headOption.getOrElse(e)
+            case (in, expected) => implicitCast(in, expected).getOrElse(in)
+          }
+        checkImplicitCastInputTypes.withNewChildren(children)
+
+      case checkExpectsInputType: ExpectsInputTypes

Review Comment:
   If all the problems come from `Concat`, shall we require all its inputs to be the same collation in the first version? It's unclear to me how useful is the pgsql semantic for concat.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1528939518


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +770,177 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {

Review Comment:
   I would appreciate some header comments of this class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1528948819


##########
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##########
@@ -41,14 +41,22 @@ class StringType private(val collationId: Int) extends AtomicType with Serializa
    */
   def isBinaryCollation: Boolean = CollationFactory.fetchCollation(collationId).isBinaryCollation
 
+  /**
+   * Returns whether the collation is indeterminate. An indeterminate collation is
+   * a result of combination of conflicting non-default implicit collations.
+   */
+  def isIndeterminateCollation: Boolean = collationId == CollationFactory.INDETERMINATE_COLLATION_ID
+
   /**
    * Type name that is shown to the customer.
    * If this is an UTF8_BINARY collation output is `string` due to backwards compatibility.
    */
   override def typeName: String =
     if (isDefaultCollation) "string"
+    else if (isIndeterminateCollation) s"string collate INDETERMINATE_COLLATION"
     else s"string collate ${CollationFactory.fetchCollation(collationId).collationName}"
 
+

Review Comment:
   remove extra line



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1529084410


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +770,177 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat

Review Comment:
   Concat is the only function for now that I know of indeterminate collations being fine as a result. That is why we are special casing here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1514266949


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion

Review Comment:
   Have looked into it. The problem we have is the test in `QueryCompilationErrorSuite`. We have 2 errors: `COLLATION_MISMATCH` and `UNSUPPORTED_COLLATION.FOR_FUNCTION`. If someone calls a function with different collations we run in to a problem which should be thrown first. Do we first try to collate parameters and then check if function is supported for that collation, or do we first throw an error that some collation is not supported by the function? If I leave the check in `StringPredicate`, the method `checkInputDataTypes` in super class is called first and we have `COLLATION_MISMATCH` first, but if I change it to the rule, we have `UNSUPPORTED_COLLATION.FOR_FUNCTION` first. The question is, which is better for the customer?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "uros-db (via GitHub)" <gi...@apache.org>.
uros-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1515617343


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -475,6 +475,24 @@
     ],
     "sqlState" : "42704"
   },
+  "COLLATION_MISMATCH" : {
+    "message" : [
+      "Could not determine which collation to use for string comparison."
+    ],
+    "subClass" : {
+      "EXPLICIT" : {
+        "message" : [
+          "Error occurred due to the mismatch between explicit collations: <explicitTypes>"

Review Comment:
   should we add an instruction for the user here? for example: `Please use the same collation for both strings.`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519576239


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -173,6 +185,8 @@ object AnsiTypeCoercion extends TypeCoercionBase {
       inType: DataType,
       expectedType: AbstractDataType): Option[DataType] = {
     (inType, expectedType) match {
+      case (_: StringType, st: StringType) =>
+        Some(st)

Review Comment:
   Can you use explicit `isDefaultCollation`? In general, let's stay away of using `StringType` case object. We left it due to backwards compatability, but we should use explicit `st: StringType if st.isDefaultCollation` checks in Spark code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519754938


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expressions to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will have.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 =>
+          throw QueryCompilationErrors
+            .explicitCollationMismatchError(
+              explicitTypes.map(t => t.asInstanceOf[StringType].typeName)
+            )
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (hasIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.indeterminateCollationError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else if (hasMultipleImplicits(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.implicitCollationMismatchError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else {
+            dataTypes.find(!_.isDefaultCollation)
+              .getOrElse(StringType)
+              .collationId
+          }
+      }
+    }
+
+    private def hasIndeterminate(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.exists(_.isIndeterminateCollation)
+
+
+    private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.filter(!_.isDefaultCollation).distinct.size > 1

Review Comment:
   At this point, where we call the method, we know we do not have any explicit collations. This should imply we only have implicit or default collations? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519594237


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expressions to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will have.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 =>
+          throw QueryCompilationErrors
+            .explicitCollationMismatchError(
+              explicitTypes.map(t => t.asInstanceOf[StringType].typeName)
+            )
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (hasIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.indeterminateCollationError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else if (hasMultipleImplicits(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.implicitCollationMismatchError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else {
+            dataTypes.find(!_.isDefaultCollation)
+              .getOrElse(StringType)
+              .collationId

Review Comment:
   I don't get this. This is the case where:
   1) We don't have inteterminate.
   2) Don'tr have multiple implicits.
   
   So, what we do is find first non default collation (?) or map to the default one?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519714112


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -138,21 +140,31 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   @scala.annotation.tailrec
   private def findWiderTypeForString(dt1: DataType, dt2: DataType): Option[DataType] = {
     (dt1, dt2) match {
-      case (StringType, _: IntegralType) => Some(LongType)
-      case (StringType, _: FractionalType) => Some(DoubleType)
-      case (StringType, NullType) => Some(StringType)
+      case (_: StringType, _: IntegralType) => Some(LongType)
+      case (_: StringType, _: FractionalType) => Some(DoubleType)
+      case (st: StringType, NullType) => Some(st)
       // If a binary operation contains interval type and string, we can't decide which
       // interval type the string should be promoted as. There are many possible interval
       // types, such as year interval, month interval, day interval, hour interval, etc.
-      case (StringType, _: AnsiIntervalType) => None
-      case (StringType, a: AtomicType) => Some(a)
-      case (other, StringType) if other != StringType => findWiderTypeForString(StringType, other)
+      case (_: StringType, _: AnsiIntervalType) => None
+      case (_: StringType, a: AtomicType) => Some(a)
+      case (other, st: StringType) if !other.isInstanceOf[StringType] =>
+        findWiderTypeForString(st, other)
       case _ => None
     }
   }
 
-  override def findWiderCommonType(types: Seq[DataType]): Option[DataType] = {
-    types.foldLeft[Option[DataType]](Some(NullType))((r, c) =>
+  override def findWiderCommonType(exprs: Seq[Expression],
+                                   failOnIndeterminate: Boolean = false): Option[DataType] = {
+    (if (exprs.map(_.dataType).partition(hasStringType)._1.distinct.size > 1) {

Review Comment:
   I am trying to see if there are at least 2 different StringTypes. In that case we need to do casting of inputs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519589124


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expressions to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will have.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 =>
+          throw QueryCompilationErrors
+            .explicitCollationMismatchError(
+              explicitTypes.map(t => t.asInstanceOf[StringType].typeName)
+            )
+        case _ =>

Review Comment:
   Maybe it would be clearer just to write `case 0` here? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547210423


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)

Review Comment:
   Let's not assume thar it's safe to only cast string-type inputs for any expression. Please follow the existing implicit cast rules and explicitly match expressions that need some of their inputs to be the same type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547826569


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)
+
+        if (hasMultipleImplicits(implicitTypes)) {
+          throw QueryCompilationErrors.implicitCollationMismatchError()
+        }
+        else {
+          implicitTypes.find(dt => !(dt == SQLConf.get.defaultStringType))
+            .getOrElse(SQLConf.get.defaultStringType)
+        }
+    }
+  }
+
+  /**
+   * This check is always preformed when we have no explicit collation. It returns true
+   * if there are more than one implicit collations. Collations are distinguished by their
+   * collationId.
+   * @param dataTypes
+   * @return
+   */
+  private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+    dataTypes.map(_.collationId)
+      .filter(dt => !(dt == SQLConf.get.defaultStringType.collationId)).distinct.size > 1
+
+}
+
+/**
+ * This rule is used to collate all existing expressions related to StringType into a single
+ * collation. Arrays are handled using their elementType and should be cast for these expressions.
+ */
+object PreCollationTypeCasts extends CollationTypeCasts {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc@(_: In
+             | _: InSubquery
+             | _: CreateArray
+             | _: ComplexTypeMergingExpression
+             | _: ArrayJoin
+             | _: BinaryExpression
+             | _: ConcatWs
+             | _: Substring) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+}
+
+/**
+ *  This rule is used for managing expressions that have possible implicit casts from different
+ *  types in ImplicitTypeCasts rule.
+ */
+object PostCollationTypeCasts extends CollationTypeCasts {

Review Comment:
   Some expressions might be cast from different types, e.g. (NullType or IntegerType) to StringType, and we want to make sure these are picked up and accordingly cast to the proper collation for expressions that lie into that category (i.e. expressions that extend ImpicitInputTypeCast and ExpectsInputType, as well as BinaryExpressions). It is either this or fine-picking these expressions in ImplicitTypeCasts, but this approach seemed more appropriate. What do you think? (We only do this for ArrayJoin, Substring and BinaryExpressions, as these are the only supported ones for collations so far)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547984669


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   Interesting. Yes array() sports an implicit collation. 'c' COLLATE UNICODE_CI sports an explicit collation.
   The rules are clear that the explicit collation wins.
   So the result is `array<STRING UNICODE_CI>` with an IMPLICIT collation.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1549237247


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType}
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, CaseWhen, Cast, Coalesce, Collate, Concat, ConcatWs, CreateArray, Expression, Greatest, If, In, InSubquery, Least, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case ifExpr: If =>
+      ifExpr.withNewChildren(
+        ifExpr.predicate +: collateToSingleType(Seq(ifExpr.trueValue, ifExpr.falseValue)))
+    case caseWhenExpr: CaseWhen =>
+      val newValues = collateToSingleType(
+        caseWhenExpr.branches.map(b => b._2) ++ caseWhenExpr.elseValue)
+      caseWhenExpr.withNewChildren(
+        interleave(Seq.empty, caseWhenExpr.branches.map(b => b._1), newValues))

Review Comment:
   Actually this exposes some gaps in this new rule
   1. We should add a trigger condition and only enter the branch if `!haveSameType(...)`
   2. We should not blindly add cast but use `castIfNotSameType`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547880103


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)

Review Comment:
   scala has `distinctBy`, we can distinct by collation id only.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1521124627


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -173,6 +185,8 @@ object AnsiTypeCoercion extends TypeCoercionBase {
       inType: DataType,
       expectedType: AbstractDataType): Option[DataType] = {
     (inType, expectedType) match {
+      case (_: StringType, st: StringType) =>
+        Some(st)

Review Comment:
   I misread the code. My bad. Everything looks good.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "uros-db (via GitHub)" <gi...@apache.org>.
uros-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1515609105


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion

Review Comment:
   I'd say `COLLATION_MISMATCH` first, `UNSUPPORTED_COLLATION.FOR_FUNCTION` second. If the user specifies COLLATION_1 for `left` and COLLATION_2 for `right`, how would we know which one to use when checking whether the functions supports this type of collation? (in this case, suppose a function supports COLLATION_1, but not COLLATION_2 - does the `UNSUPPORTED_COLLATION.FOR_FUNCTION` pass or fail?)
   
   Hence, I think we would first need to establish that COLLATION_1 and COLLATION_2 are the same (no `COLLATION_MISMATCH `), before checking whether the function supports the requested collation (no `UNSUPPORTED_COLLATION.FOR_FUNCTION`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519587302


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expressions to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will have.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct

Review Comment:
   Again, check collation id explicitly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519598303


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,94 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expressions to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will have.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 =>
+          throw QueryCompilationErrors
+            .explicitCollationMismatchError(
+              explicitTypes.map(t => t.asInstanceOf[StringType].typeName)
+            )
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (hasIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.indeterminateCollationError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else if (hasMultipleImplicits(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.implicitCollationMismatchError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else {
+            dataTypes.find(!_.isDefaultCollation)
+              .getOrElse(StringType)
+              .collationId
+          }
+      }
+    }
+
+    private def hasIndeterminate(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.exists(_.isIndeterminateCollation)
+
+
+    private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.filter(!_.isDefaultCollation).distinct.size > 1

Review Comment:
   Hm, is `isDefaultCollation` sufficient check to determine whether this is a non-implicit collation?
   Can you also add a comment describint what is definition of implicit and explicit collations?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on PR #45383:
URL: https://github.com/apache/spark/pull/45383#issuecomment-1999140950

   > > Some high-level questions:
   > > 
   > > 1. If a function requires certain collations but the input uses a different collation, shall we implicitly cast or fail?
   > > 2. If a function's inputs do not use the same collation, shall we implicit cast or fail?
   > > 3. If we cast a string with collation to integer or datetime, do we need to consider the collation?
   > 
   > @cloud-fan - these are great questions and I think that they should be part of the spec. Rough answers from my side are:
   > 
   > 1. I think that we should fail but I think that there are some subtle caveats here that should be covered in the design spec.
   > 2. Depends on a function. e.g. for contains we should fail. For concat we should succeed.
   > 3. No. Decimal/datetime formatting should be part of "language settings" which are not part of collation track.
   > 
   > @mihailom-db will extend casting section of the doc.
   @cloud-fan 
   
   We will have a meeting with Serge today to discuss some of these questions to make sure we got everything right, but for now, my view is this:
   
   1. We should fail, will extend this part after meeting with Serge. Although, apart from lockdown that @uros-db is working on, no function for now supports specific collation.
   2. @dbatomic I partially agree. By design doc we should always try to cast to same type. This is more of a question if we have some conflicts of implicit types or multiple explicit types. If this conflict happens then contains fails, and concat doesn't, but we still try to cast.
   3. Agreed on this point.
   
   Also, we will have a meeting with Serge today to discuss some of these questions to make sure we got everything right.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530220554


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -609,8 +610,10 @@ abstract class TypeCoercionBase {
       case c @ Concat(children) if conf.concatBinaryAsString ||
         !children.map(_.dataType).forall(_ == BinaryType) =>
         val newChildren = c.children.map { e =>
-          implicitCast(e, StringType).getOrElse(e)
+          if (e.dataType.isInstanceOf[StringType]) e
+          else implicitCast(e, StringType).getOrElse(e)

Review Comment:
   We should update `def implicitCast` if we don't want to allow implicit cast `_: StringType` to `StringType`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547273755


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,26 +707,39 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)
+        }
+        val st = getOutputCollation(e.children)
+        val children: Seq[Expression] = childrenBeforeCollations.map {

Review Comment:
   Once again, this is the problem, we need to call getOutputCollation before we call implicitCast, as implicitCast only concerns one child, and not all children as a group. For collations we need to be sure we decided on the right collation first and then enter a casting rule. But if we do this, we would need to pass additional stuff to implicitCast, which changes the logic of the core function. Another idea was to implement implicitCasting for collations solely in a new rule, but we would end up copying a lot of already existent code in the implicitCast for TypeCollections and StringTypes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546420774


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   Explicit collation NEVER propagates. It only applies to the immediate consumer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546370140


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   This might be a good question for @srielau. How do we want this query to behave: `SELECT array('a', 'b' collate UNICODE) || ('c' collate UNICODE_CI)` (Concat_Ws in our implementation)? This query in Postgre fails with explicit type mismatch. Do we want to propagate explicit collations from elements to arrays. I would say yes, as ArrayType is constructed with a parameter called `elementType`. This suggests, if elements are explicit, then ArrayType should be as well. Also, on other operations with arrays, as we defined that function outputs are implicit, I would say array creation is then the only place where we should propagate this explicit meaning.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546283459


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>

Review Comment:
   Can you check all the implicit cast rules and see if we miss to handle any expression?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546445756


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   This is a good news. Let's not special-case CreateArray, otherwise people may question about CreateMap and CreateStruct. They are just SQL functions, not a special building block.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546397910


##########
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##########
@@ -27,7 +27,9 @@ import org.apache.spark.sql.catalyst.util.CollationFactory
  * @param collationId The id of collation for this StringType.
  */
 @Stable
-class StringType private(val collationId: Int) extends AtomicType with Serializable {
+class StringType private(val collationId: Int, var isExplicit: Boolean = false)

Review Comment:
   Yeah, good point, needed it to be var when it was not in the constructor. Changing it to val.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1544055842


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    // Case when we do fail if resulting collation is indeterminate

Review Comment:
   I thought we excluded indeterminate collation from this PR?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546396569


##########
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##########
@@ -27,7 +27,9 @@ import org.apache.spark.sql.catalyst.util.CollationFactory
  * @param collationId The id of collation for this StringType.
  */
 @Stable
-class StringType private(val collationId: Int) extends AtomicType with Serializable {
+class StringType private(val collationId: Int, var isExplicit: Boolean = false)

Review Comment:
   why it's a `var`? does it need to be mutable?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547818283


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)

Review Comment:
   nit:
   ```
   val implicitCollations =
     ...
     .distinct
   if (implicitCollations > 1) {
     throw ...
   } else {
     implicitCollations.headOption.getOrElse(SQLConf.get.defaultStringType)
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547818939


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,16 +705,17 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val children: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)

Review Comment:
   please revert these cosmetic changes.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,16 +705,17 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val children: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)
         }
         e.withNewChildren(children)
 
       case e: ExpectsInputTypes if e.inputTypes.nonEmpty =>
         // Convert NullType into some specific target type for ExpectsInputTypes that don't do
         // general implicit casting.
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val children: Seq[Expression] =
+          e.children.zip(e.inputTypes).map { case (in, expected) =>

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547841866


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)
+
+        if (hasMultipleImplicits(implicitTypes)) {
+          throw QueryCompilationErrors.implicitCollationMismatchError()
+        }
+        else {
+          implicitTypes.find(dt => !(dt == SQLConf.get.defaultStringType))
+            .getOrElse(SQLConf.get.defaultStringType)
+        }
+    }
+  }
+
+  /**
+   * This check is always preformed when we have no explicit collation. It returns true
+   * if there are more than one implicit collations. Collations are distinguished by their
+   * collationId.
+   * @param dataTypes
+   * @return
+   */
+  private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+    dataTypes.map(_.collationId)
+      .filter(dt => !(dt == SQLConf.get.defaultStringType.collationId)).distinct.size > 1
+
+}
+
+/**
+ * This rule is used to collate all existing expressions related to StringType into a single
+ * collation. Arrays are handled using their elementType and should be cast for these expressions.
+ */
+object PreCollationTypeCasts extends CollationTypeCasts {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc@(_: In
+             | _: InSubquery
+             | _: CreateArray
+             | _: ComplexTypeMergingExpression
+             | _: ArrayJoin
+             | _: BinaryExpression
+             | _: ConcatWs
+             | _: Substring) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+}
+
+/**
+ *  This rule is used for managing expressions that have possible implicit casts from different
+ *  types in ImplicitTypeCasts rule.
+ */
+object PostCollationTypeCasts extends CollationTypeCasts {

Review Comment:
   Can't `PreCollationTypeCasts` alone do the work?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #45383:
URL: https://github.com/apache/spark/pull/45383#issuecomment-2035818786

   the yarn and docker test failures are unrelated, thanks, merging to master!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547811709


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)
+
+        if (hasMultipleImplicits(implicitTypes)) {
+          throw QueryCompilationErrors.implicitCollationMismatchError()
+        }
+        else {
+          implicitTypes.find(dt => !(dt == SQLConf.get.defaultStringType))
+            .getOrElse(SQLConf.get.defaultStringType)
+        }
+    }
+  }
+
+  /**
+   * This check is always preformed when we have no explicit collation. It returns true
+   * if there are more than one implicit collations. Collations are distinguished by their
+   * collationId.
+   * @param dataTypes
+   * @return
+   */
+  private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+    dataTypes.map(_.collationId)
+      .filter(dt => !(dt == SQLConf.get.defaultStringType.collationId)).distinct.size > 1
+
+}
+
+/**
+ * This rule is used to collate all existing expressions related to StringType into a single
+ * collation. Arrays are handled using their elementType and should be cast for these expressions.
+ */
+object PreCollationTypeCasts extends CollationTypeCasts {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc@(_: In
+             | _: InSubquery
+             | _: CreateArray
+             | _: ComplexTypeMergingExpression

Review Comment:
   let's narrow down the scope. Not all `ComplexTypeMergingExpression` expressions require all its inputs to be the same type. Please follow existing implicit cast rules and match If, CaseWhen, etc. explicitly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "uros-db (via GitHub)" <gi...@apache.org>.
uros-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1515609105


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion

Review Comment:
   I'd say `COLLATION_MISMATCH` first, `UNSUPPORTED_COLLATION.FOR_FUNCTION` second. If the user specifies COLLATION_1 for `left` and COLLATION_2 for `right`, how would we know which one to use when checking whether the functions supports this type of collation? (in this case, suppose a function supports COLLATION_1, but not COLLATION_2 - does the `UNSUPPORTED_COLLATION.FOR_FUNCTION` pass or fail?)
   
   Hence, I think we would first need to establish that COLLATION_1 and COLLATION_2 are the same (no `COLLATION_MISMATCH`), before checking whether the function supports the requested collation (no `UNSUPPORTED_COLLATION.FOR_FUNCTION`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513035016


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will be.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 => throw QueryCompilationErrors.explicitCollationMismatchError(
+          explicitTypes.head.simpleString, explicitTypes.tail.head.simpleString)
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (isIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.indeterminateCollationError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else if (hasMultipleImplicits(dataTypes)) {
+            if (failOnIndeterminate) {
+              throw QueryCompilationErrors.implicitCollationMismatchError()
+            } else {
+              CollationFactory.INDETERMINATE_COLLATION_ID
+            }
+          }
+          else {
+            dataTypes.find(!_.isDefaultCollation)
+              .getOrElse(StringType)
+              .collationId
+          }
+      }
+    }
+
+    private def isIndeterminate(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.exists(_.isIndeterminateCollation)
+
+
+    private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+      dataTypes.filter(!_.isDefaultCollation).distinct.size > 1
+
+    private def hasExplicitCollation(expression: Expression): Boolean = {
+      if (!expression.dataType.isInstanceOf[StringType]) {
+        false
+      }
+      else {
+        expression match {
+          case _: Collate => true

Review Comment:
   ```suggestion
             case Collate => true
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513022131


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -475,6 +475,24 @@
     ],
     "sqlState" : "42704"
   },
+  "COLLATION_MISMATCH" : {
+    "message" : [
+      "Could not determine which collation to use for string comparison."
+    ],
+    "subClass" : {
+      "EXPLICIT" : {
+        "message" : [
+          "Error occurred due to the mismatch between explicit collations \"<left>\" and \"<right>\""
+        ]
+      },
+      "IMPLICIT" : {
+        "message" : [
+          "Error occurred due to the mismatch between multiple implicit collations. Use COLLATE function to set the collation explicitly."

Review Comment:
   ```suggestion
             "Error occurred due to the mismatch between multiple implicit non-default collations. Use COLLATE function to set the collation explicitly."
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513061821


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion

Review Comment:
   why not just add `StringPredicate` to the rule like we did for `BinaryComparison`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513941920


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -958,14 +1062,16 @@ object TypeCoercion extends TypeCoercionBase {
 
   override def implicitCast(e: Expression, expectedType: AbstractDataType): Option[Expression] = {
     implicitCast(e.dataType, expectedType).map { dt =>
-      if (dt == e.dataType) e else Cast(e, dt)
+      if (dt == e.dataType) { e }

Review Comment:
   I changed it for debugging, as I wanted to see which branch I jump into, will change it back.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530226268


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +770,177 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat

Review Comment:
   why is it fine? What should be the return type of `Concat` if its input have different collations?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1528926135


##########
python/pyspark/sql/types.py:
##########
@@ -262,8 +262,8 @@ def __init__(self, collationId: int = 0):
     def collationIdToName(self) -> str:
         return (
             " collate %s" % StringType.collationNames[self.collationId]

Review Comment:
   this is kind of unreadable? Maybe go with:
   ```python
   if self.collationId == 0: ""
   elif self.collationId == -1: "INTERMINATE"
   else: StringType.collationNames[self.collationId]
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1544095362


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    // Case when we do fail if resulting collation is indeterminate
+    case checkCastWithoutIndeterminate @ (_: BinaryExpression
+                                          | _: Predicate
+                                          | _: SortOrder
+                                          | _: ExpectsInputTypes
+                                          | _: ComplexTypeMergingExpression
+                                          | _: CreateArray)
+      if shouldCast(checkCastWithoutIndeterminate.children) =>
+      val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+      checkCastWithoutIndeterminate.withNewChildren(newChildren)
+  }
+
+  def shouldCast(types: Seq[Expression]): Boolean = {
+    types.filter(e => hasStringType(e.dataType))
+      .map(e => extractStringType(e.dataType).collationId).distinct.size > 1
+  }
+
+  /**
+   * Checks whether given data type contains StringType.
+   */
+  @tailrec
+  def hasStringType(dt: DataType): Boolean = dt match {
+    case _: StringType => true
+    case ArrayType(et, _) => hasStringType(et)
+    case _ => false
+  }
+
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, collationId: Int): Option[Expression] =
+    castStringType(expr.dataType, collationId).map { dt =>
+      if (dt == expr.dataType) expr else Cast(expr, dt)
+    }
+
+  private def castStringType(inType: AbstractDataType, collationId: Int): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId == collationId => st
+      case _: StringType => StringType(collationId)
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, collationId).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val collationId = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, collationId).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(exprs: Seq[Expression]): Int = {
+    val explicitTypes = exprs.filter(hasExplicitCollation)
+      .map(e => extractStringType(e.dataType).collationId).distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => explicitTypes.head
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val dataTypes = exprs.filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))
+
+        if (hasMultipleImplicits(dataTypes)) {
+          throw QueryCompilationErrors.implicitCollationMismatchError()
+        }
+        else {
+          dataTypes.find(dt => !(dt == SQLConf.get.defaultStringType))
+            .getOrElse(SQLConf.get.defaultStringType)
+            .collationId
+        }
+    }
+  }
+
+  /**
+   * This check is always preformed when we have no explicit collation. It returns true
+   * if there are more than one implicit collations. Collations are distinguished by their
+   * collationId.
+   * @param dataTypes
+   * @return
+   */
+  private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+    dataTypes.filter(dt => !(dt == SQLConf.get.defaultStringType))
+      .map(_.collationId).distinct.size > 1
+
+  /**
+   * Checks if a given expression has explicitly set collation. For complex DataTypes
+   * we need to check nested children.
+   * @param expression
+   * @return
+   */
+  private def hasExplicitCollation(expression: Expression): Boolean = {
+    expression match {
+      case _: Collate => true
+      case e if e.dataType.isInstanceOf[ArrayType]
+      => expression.children.exists(hasExplicitCollation)

Review Comment:
   The exact definition of explicit collation is to if there is a explicit use of COLLATE keyword to set collation of a parameter. In this case, when we have arrays, a query `SELECT array('b' collate unicode_ci) || 'a' collate unicode;` (for us concat_ws) fails in postgreSQL with explicit collation mismatch which requires us to  check nested elements of arrayType for explicitly collated strings.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530224691


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala:
##########
@@ -509,18 +509,10 @@ abstract class StringPredicate extends BinaryExpression
       return checkResult
     }
     // Additional check needed for collation compatibility
-    val rightCollationId: Int = right.dataType.asInstanceOf[StringType].collationId
-    if (collationId != rightCollationId) {
-      DataTypeMismatch(
-        errorSubClass = "COLLATION_MISMATCH",
-        messageParameters = Map(
-          "collationNameLeft" -> CollationFactory.fetchCollation(collationId).collationName,
-          "collationNameRight" -> CollationFactory.fetchCollation(rightCollationId).collationName
-        )
-      )
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
+    val outputCollationId: Int = TypeCoercion
+      .CollationTypeCasts
+      .getOutputCollation(Seq(left, right))

Review Comment:
   It looks weird to have this in the type check function. Will it throw exception?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530235790


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,184 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat
+        if shouldCast(checkCastWithIndeterminate.children) =>
+        val newChildren =
+          collateToSingleType(checkCastWithIndeterminate.children, failOnIndeterminate = false)
+        checkCastWithIndeterminate.withNewChildren(newChildren)
+
+      case checkCastWithoutIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if shouldCast(checkCastWithoutIndeterminate.children) =>
+        val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+        checkCastWithoutIndeterminate.withNewChildren(newChildren)
+
+      case checkIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if hasIndeterminate(checkIndeterminate.children
+          .filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))) =>
+        throw QueryCompilationErrors.indeterminateCollationError()
+
+      case checkImplicitCastInputTypes: ImplicitCastInputTypes
+        if checkImplicitCastInputTypes.children.exists(e => hasStringType(e.dataType))
+          && checkImplicitCastInputTypes.inputTypes.nonEmpty =>
+        val collationId: Int =
+          getOutputCollation(checkImplicitCastInputTypes
+            .children.filter { e => hasStringType(e.dataType) })
+        val children: Seq[Expression] = checkImplicitCastInputTypes
+          .children.zip(checkImplicitCastInputTypes.inputTypes).map {
+            case (e, st) if hasStringType(st) =>
+              castStringType(e, collationId, Some(st)).getOrElse(e)
+            case (e, TypeCollection(types)) if types.exists(hasStringType) =>
+              types.flatMap{ dt =>
+                if (hasStringType(dt)) {
+                  castStringType(e, collationId, Some(dt))
+                } else {
+                  implicitCast(e, dt)
+                }
+              }.headOption.getOrElse(e)
+            case (in, expected) => implicitCast(in, expected).getOrElse(in)
+          }
+        checkImplicitCastInputTypes.withNewChildren(children)
+
+      case checkExpectsInputType: ExpectsInputTypes

Review Comment:
   I don't think we should handle implicit cast in this new rule. The new rule should only resolve conflicting string collations for string function inputs. We should try our best to not do one thing in multiple places and rely on the rule execution order. It's much easier to reason about if we only have one central place to do implicit cast.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1519766425


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -173,6 +185,8 @@ object AnsiTypeCoercion extends TypeCoercionBase {
       inType: DataType,
       expectedType: AbstractDataType): Option[DataType] = {
     (inType, expectedType) match {
+      case (_: StringType, st: StringType) =>
+        Some(st)

Review Comment:
   I am not sure I get this. We need to add the rule for casting any StringType class as acceptsType below would just lend us into keeping original collationId, and we would not be able to cast different collations into one.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1533863046


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,184 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat
+        if shouldCast(checkCastWithIndeterminate.children) =>
+        val newChildren =
+          collateToSingleType(checkCastWithIndeterminate.children, failOnIndeterminate = false)
+        checkCastWithIndeterminate.withNewChildren(newChildren)
+
+      case checkCastWithoutIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if shouldCast(checkCastWithoutIndeterminate.children) =>
+        val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+        checkCastWithoutIndeterminate.withNewChildren(newChildren)
+
+      case checkIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if hasIndeterminate(checkIndeterminate.children
+          .filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))) =>
+        throw QueryCompilationErrors.indeterminateCollationError()
+
+      case checkImplicitCastInputTypes: ImplicitCastInputTypes
+        if checkImplicitCastInputTypes.children.exists(e => hasStringType(e.dataType))
+          && checkImplicitCastInputTypes.inputTypes.nonEmpty =>
+        val collationId: Int =
+          getOutputCollation(checkImplicitCastInputTypes
+            .children.filter { e => hasStringType(e.dataType) })
+        val children: Seq[Expression] = checkImplicitCastInputTypes
+          .children.zip(checkImplicitCastInputTypes.inputTypes).map {
+            case (e, st) if hasStringType(st) =>
+              castStringType(e, collationId, Some(st)).getOrElse(e)
+            case (e, TypeCollection(types)) if types.exists(hasStringType) =>
+              types.flatMap{ dt =>
+                if (hasStringType(dt)) {
+                  castStringType(e, collationId, Some(dt))
+                } else {
+                  implicitCast(e, dt)
+                }
+              }.headOption.getOrElse(e)
+            case (in, expected) => implicitCast(in, expected).getOrElse(in)
+          }
+        checkImplicitCastInputTypes.withNewChildren(children)
+
+      case checkExpectsInputType: ExpectsInputTypes
+        if checkExpectsInputType.children.exists(e => hasStringType(e.dataType))
+          && checkExpectsInputType.inputTypes.nonEmpty =>
+          val collationId: Int = getOutputCollation(
+            checkExpectsInputType.children.filter {e => hasStringType(e.dataType)})
+          val children: Seq[Expression] = checkExpectsInputType
+            .children.zip(checkExpectsInputType.inputTypes).map {
+            case (st, _) if hasStringType(st.dataType) =>
+              castStringType(st, collationId).getOrElse(st)
+            case (nt, e)
+              if hasStringType(e) && nt.dataType == NullType =>
+              castStringType(nt, collationId, Some(e)).getOrElse(nt)
+            case (nt, e: TypeCollection)
+              if hasStringType(e.defaultConcreteType) && nt.dataType == NullType =>
+              castStringType(nt, collationId, Some(e.defaultConcreteType)).getOrElse(nt)
+            case (in, _) => in
+            }
+          checkExpectsInputType.withNewChildren(children)
+    }
+
+    def shouldCast(types: Seq[Expression]): Boolean = {
+      types.filter(e => hasStringType(e.dataType))
+        .map(e => extractStringType(e.dataType).collationId).distinct.size > 1
+    }
+
+    /**
+     * Whether the data type contains StringType.
+     */
+    @tailrec

Review Comment:
   This is not possible for now. I think this PR is already too big and if I change it to this I need to deal with StructType and MapType. I suggest we leave those two for follow-up PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION][WIP] Implicit casting on collated expressions [spark]

Posted by "stefankandic (via GitHub)" <gi...@apache.org>.
stefankandic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1513038768


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +782,91 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override def transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case b @ BinaryComparison(left, right) if shouldCast(Seq(left.dataType, right.dataType)) =>
+        val newChildren = collateToSingleType(Seq(left, right))
+        b.withNewChildren(newChildren)
+    }
+
+    def shouldCast(types: Seq[DataType]): Boolean = {
+      types.forall(_.isInstanceOf[StringType]) && types.distinct.length > 1
+    }
+
+    /**
+     *  Collates the input expression to a single collation.
+     */
+    def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+      val collationId = getOutputCollation(exprs)
+
+      exprs.map { expression =>
+        expression.dataType match {
+          case st: StringType if st.collationId == collationId =>
+            expression
+          case _: StringType =>
+            Cast(expression, StringType(collationId))
+        }
+      }
+    }
+
+    /**
+     * Based on the data types of the input expressions this method determines
+     * a collation type which the output will be.
+     */
+    def getOutputCollation(exprs: Seq[Expression], failOnIndeterminate: Boolean = true): Int = {
+      val explicitTypes = exprs.filter(hasExplicitCollation).map(_.dataType).distinct
+
+      explicitTypes.size match {
+        case 1 => explicitTypes.head.asInstanceOf[StringType].collationId
+        case size if size > 1 => throw QueryCompilationErrors.explicitCollationMismatchError(
+          explicitTypes.head.simpleString, explicitTypes.tail.head.simpleString)
+        case _ =>
+          val dataTypes = exprs.map(_.dataType.asInstanceOf[StringType])
+
+          if (isIndeterminate(dataTypes)) {
+            if (failOnIndeterminate) {

Review Comment:
   this can be a private method



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on PR #45383:
URL: https://github.com/apache/spark/pull/45383#issuecomment-2024740857

   Update: Removed introduction of indeterminate collations. I will create a separate ticket for support of indeterminate collation and will do it as a followup. I did not remove error-class addition, as it will be used and it does not pose problem for the code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1537186083


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -182,16 +184,19 @@ object AnsiTypeCoercion extends TypeCoercionBase {
 
       // If a function expects a StringType, no StringType instance should be implicitly cast to
       // StringType with a collation that's not accepted (aka. lockdown unsupported collations).
-      case (_: StringType, StringType) => None
+      case (_: StringType, _: StringType) => None
       case (_: StringType, _: StringTypeCollated) => None
 
+      case (DateType, AnyTimestampType) =>

Review Comment:
   Let me check, this is a problem I needed to solve as a conflict with another PR, which simplified this function



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1544100458


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala:
##########
@@ -112,12 +112,14 @@ case class Collate(child: Expression, collationName: String)
   since = "4.0.0",
   group = "string_funcs")
 // scalastyle:on line.contains.tab
-case class Collation(child: Expression) extends UnaryExpression with RuntimeReplaceable {
-  override def dataType: DataType = StringType
+case class Collation(child: Expression)
+  extends UnaryExpression with RuntimeReplaceable with ExpectsInputTypes {
+  override def dataType: DataType = child.dataType

Review Comment:
   I was thinking of some cases when indeterminate comes. What if our costumer wants to concat the collation used to their strings and filter on something like that. If we return a StringType, we would face a problem of getting indeterminate collation failing to keep the pass through for our functions. This is more of a problem if users set default collation to be something else, so we might say that changing return type to default collation would be the better choice?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547288333


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -702,26 +707,39 @@ abstract class TypeCoercionBase {
         }.getOrElse(b)  // If there is no applicable conversion, leave expression unchanged.
 
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
-        val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) =>
+        val childrenBeforeCollations: Seq[Expression] = e.children.zip(e.inputTypes).map {
           // If we cannot do the implicit cast, just use the original input.
-          implicitCast(in, expected).getOrElse(in)
+          case (in, expected) => implicitCast(in, expected).getOrElse(in)
+        }
+        val st = getOutputCollation(e.children)
+        val children: Seq[Expression] = childrenBeforeCollations.map {

Review Comment:
   Note: if we want to have correlation between function inputs, please match the expression explicitly to do so. `ImplicitCastInputTypes` does not indicate input correlation and please DO NOT make such assumptions here.
   
   Looking at `ConcatWs`, it does need the inputs to use the same string collation. Let's match it in the new rule and deal with it correctly.
   
   In principle, let's not generalize things without seeing the full picture. What we should do to `ConcatWs` does not necessarily apply to all `ImplicitCastInputTypes` implementations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546413940


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   How I understood it, arrays are bulding blocks for types as well. The difference is that arrays have to be built from some different types which makes them complex. But `substring`, `concat`, etc. are functions that return something based on a transformation behaviour. In other words, CreateArray is not even considering the the values, it only builds a new 'basic' block. That is why I am also concerned with what we want to do with Elt, do we want to do casting or just propagation of inputs, as it does not change any string, but only takes a specific value from the given index. But again, this differs from array access, as arrays need to have a single type for all input expressions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546285091


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>
+      val newChildren = collateToSingleType(pesc.children, true)
+      pesc.withNewChildren(newChildren)
+  }
+
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, st: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case ost: StringType if ost.collationId == st.collationId
+        && ost.isExplicit == st.isExplicit => null
+      case _: StringType => st
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, st).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression],
+                          preserveExplicit: Boolean = false): Seq[Expression] = {

Review Comment:
   nit 4 spaces indentation
   ```
   def func
       para1: xxx,
       para2: xxx,
       ...) ...
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547309134


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)

Review Comment:
   This is the problem. If I understood correctly, we do not want to replicate the code from ImplicitTypeCasts. This rule is concerned with transforming datatypes into their expected DataTypes, but CollationTypeCasts is concerned with transforming StringTypes into their expected collated StringType. There is a difference, as collated StringType is calculated based on expression parameters that have StringTypes not on what someone expects the input types to be. As you mentioned for `If`, this new collation type rule does not want to fail instead of the IfCoercion which fails if it cannot find wider for left and right, but this new rule wants to fail and mitigate errors of a customer providing differently collated strings.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1549347237


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType}
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, CaseWhen, Cast, Coalesce, Collate, Concat, ConcatWs, CreateArray, Expression, Greatest, If, In, InSubquery, Least, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case ifExpr: If =>
+      ifExpr.withNewChildren(
+        ifExpr.predicate +: collateToSingleType(Seq(ifExpr.trueValue, ifExpr.falseValue)))
+    case caseWhenExpr: CaseWhen =>
+      val newValues = collateToSingleType(
+        caseWhenExpr.branches.map(b => b._2) ++ caseWhenExpr.elseValue)
+      caseWhenExpr.withNewChildren(
+        interleave(Seq.empty, caseWhenExpr.branches.map(b => b._1), newValues))

Review Comment:
   We actually do not blindly Cast in CollationTypeCasts. We only cast if we get different collations, as all other cases go to null branch in castStringType. Maybe it is better to keep this check internal to this rule, as we will add the priority flag and we will need to handle casting of priority in this rule as well later. Am adding the haveSameType for now to make it check the input types, but will change this code in the following PR to include priorities in the internal implementation of haveSameType check.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1549233784


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType}
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, CaseWhen, Cast, Coalesce, Collate, Concat, ConcatWs, CreateArray, Expression, Greatest, If, In, InSubquery, Least, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case ifExpr: If =>
+      ifExpr.withNewChildren(
+        ifExpr.predicate +: collateToSingleType(Seq(ifExpr.trueValue, ifExpr.falseValue)))
+    case caseWhenExpr: CaseWhen =>
+      val newValues = collateToSingleType(
+        caseWhenExpr.branches.map(b => b._2) ++ caseWhenExpr.elseValue)
+      caseWhenExpr.withNewChildren(
+        interleave(Seq.empty, caseWhenExpr.branches.map(b => b._1), newValues))

Review Comment:
   The code looks a bit complicated now. Can we follow the existing rule?
   ```
     object CaseWhenCoercion extends TypeCoercionRule {
       override val transform: PartialFunction[Expression, Expression] = {
         case c: CaseWhen if c.childrenResolved && !haveSameType(c.inputTypesForMerging) =>
           val maybeCommonType = findWiderCommonType(c.inputTypesForMerging)
           maybeCommonType.map { commonType =>
             val newBranches = c.branches.map { case (condition, value) =>
               (condition, castIfNotSameType(value, commonType))
             }
             val newElseValue = c.elseValue.map(castIfNotSameType(_, commonType))
             CaseWhen(newBranches, newElseValue)
           }.getOrElse(c)
       }
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530301512


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,184 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat
+        if shouldCast(checkCastWithIndeterminate.children) =>
+        val newChildren =
+          collateToSingleType(checkCastWithIndeterminate.children, failOnIndeterminate = false)
+        checkCastWithIndeterminate.withNewChildren(newChildren)
+
+      case checkCastWithoutIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if shouldCast(checkCastWithoutIndeterminate.children) =>
+        val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+        checkCastWithoutIndeterminate.withNewChildren(newChildren)
+
+      case checkIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if hasIndeterminate(checkIndeterminate.children
+          .filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))) =>
+        throw QueryCompilationErrors.indeterminateCollationError()
+
+      case checkImplicitCastInputTypes: ImplicitCastInputTypes
+        if checkImplicitCastInputTypes.children.exists(e => hasStringType(e.dataType))
+          && checkImplicitCastInputTypes.inputTypes.nonEmpty =>
+        val collationId: Int =
+          getOutputCollation(checkImplicitCastInputTypes
+            .children.filter { e => hasStringType(e.dataType) })
+        val children: Seq[Expression] = checkImplicitCastInputTypes
+          .children.zip(checkImplicitCastInputTypes.inputTypes).map {
+            case (e, st) if hasStringType(st) =>
+              castStringType(e, collationId, Some(st)).getOrElse(e)
+            case (e, TypeCollection(types)) if types.exists(hasStringType) =>
+              types.flatMap{ dt =>
+                if (hasStringType(dt)) {
+                  castStringType(e, collationId, Some(dt))
+                } else {
+                  implicitCast(e, dt)
+                }
+              }.headOption.getOrElse(e)
+            case (in, expected) => implicitCast(in, expected).getOrElse(in)
+          }
+        checkImplicitCastInputTypes.withNewChildren(children)
+
+      case checkExpectsInputType: ExpectsInputTypes

Review Comment:
   I agree and could move implicitCasting from here to `ImpicitTypeCast`. Only problem is that we then have to change multiple checks to run `getOutputCollation` (e.g. `ConcatCoercion`) as they might cast to StringType(0) even though we want collated cast. I was thinking about performance issues if this is ok to run check for collationId multiple times in different checks? Do you have some idea on how heavy this operation would be to the whole casting process, as I am not still quite sure?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530367223


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,184 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat
+        if shouldCast(checkCastWithIndeterminate.children) =>
+        val newChildren =
+          collateToSingleType(checkCastWithIndeterminate.children, failOnIndeterminate = false)
+        checkCastWithIndeterminate.withNewChildren(newChildren)
+
+      case checkCastWithoutIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if shouldCast(checkCastWithoutIndeterminate.children) =>
+        val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+        checkCastWithoutIndeterminate.withNewChildren(newChildren)
+
+      case checkIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if hasIndeterminate(checkIndeterminate.children
+          .filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))) =>
+        throw QueryCompilationErrors.indeterminateCollationError()
+
+      case checkImplicitCastInputTypes: ImplicitCastInputTypes
+        if checkImplicitCastInputTypes.children.exists(e => hasStringType(e.dataType))
+          && checkImplicitCastInputTypes.inputTypes.nonEmpty =>
+        val collationId: Int =
+          getOutputCollation(checkImplicitCastInputTypes
+            .children.filter { e => hasStringType(e.dataType) })
+        val children: Seq[Expression] = checkImplicitCastInputTypes
+          .children.zip(checkImplicitCastInputTypes.inputTypes).map {
+            case (e, st) if hasStringType(st) =>
+              castStringType(e, collationId, Some(st)).getOrElse(e)
+            case (e, TypeCollection(types)) if types.exists(hasStringType) =>
+              types.flatMap{ dt =>
+                if (hasStringType(dt)) {
+                  castStringType(e, collationId, Some(dt))
+                } else {
+                  implicitCast(e, dt)
+                }
+              }.headOption.getOrElse(e)
+            case (in, expected) => implicitCast(in, expected).getOrElse(in)
+          }
+        checkImplicitCastInputTypes.withNewChildren(children)
+
+      case checkExpectsInputType: ExpectsInputTypes

Review Comment:
   Unfortunately, we do not have a trait to indicate the inputs of an expression should all be the same data type. We have various type coercion rules to do it for various expressions: `InConversion`, `ConcatCoercion`, `IfCoercion`, etc.
   
   This is probably not a big problem, as not many functions support more than one string input and we can add a new rule to merge string collations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1530286778


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##########
@@ -764,6 +773,184 @@ abstract class TypeCoercionBase {
     }
   }
 
+  object CollationTypeCasts extends TypeCoercionRule {
+    override val transform: PartialFunction[Expression, Expression] = {
+      case e if !e.childrenResolved => e
+
+      case checkCastWithIndeterminate: Concat
+        if shouldCast(checkCastWithIndeterminate.children) =>
+        val newChildren =
+          collateToSingleType(checkCastWithIndeterminate.children, failOnIndeterminate = false)
+        checkCastWithIndeterminate.withNewChildren(newChildren)
+
+      case checkCastWithoutIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if shouldCast(checkCastWithoutIndeterminate.children) =>
+        val newChildren = collateToSingleType(checkCastWithoutIndeterminate.children)
+        checkCastWithoutIndeterminate.withNewChildren(newChildren)
+
+      case checkIndeterminate@(_: BinaryExpression | _: In | _: SortOrder)
+        if hasIndeterminate(checkIndeterminate.children
+          .filter(e => hasStringType(e.dataType))
+          .map(e => extractStringType(e.dataType))) =>
+        throw QueryCompilationErrors.indeterminateCollationError()
+
+      case checkImplicitCastInputTypes: ImplicitCastInputTypes
+        if checkImplicitCastInputTypes.children.exists(e => hasStringType(e.dataType))
+          && checkImplicitCastInputTypes.inputTypes.nonEmpty =>
+        val collationId: Int =
+          getOutputCollation(checkImplicitCastInputTypes
+            .children.filter { e => hasStringType(e.dataType) })
+        val children: Seq[Expression] = checkImplicitCastInputTypes
+          .children.zip(checkImplicitCastInputTypes.inputTypes).map {
+            case (e, st) if hasStringType(st) =>
+              castStringType(e, collationId, Some(st)).getOrElse(e)
+            case (e, TypeCollection(types)) if types.exists(hasStringType) =>
+              types.flatMap{ dt =>
+                if (hasStringType(dt)) {
+                  castStringType(e, collationId, Some(dt))
+                } else {
+                  implicitCast(e, dt)
+                }
+              }.headOption.getOrElse(e)
+            case (in, expected) => implicitCast(in, expected).getOrElse(in)
+          }
+        checkImplicitCastInputTypes.withNewChildren(children)
+
+      case checkExpectsInputType: ExpectsInputTypes
+        if checkExpectsInputType.children.exists(e => hasStringType(e.dataType))
+          && checkExpectsInputType.inputTypes.nonEmpty =>
+          val collationId: Int = getOutputCollation(
+            checkExpectsInputType.children.filter {e => hasStringType(e.dataType)})
+          val children: Seq[Expression] = checkExpectsInputType
+            .children.zip(checkExpectsInputType.inputTypes).map {
+            case (st, _) if hasStringType(st.dataType) =>
+              castStringType(st, collationId).getOrElse(st)
+            case (nt, e)
+              if hasStringType(e) && nt.dataType == NullType =>
+              castStringType(nt, collationId, Some(e)).getOrElse(nt)
+            case (nt, e: TypeCollection)
+              if hasStringType(e.defaultConcreteType) && nt.dataType == NullType =>
+              castStringType(nt, collationId, Some(e.defaultConcreteType)).getOrElse(nt)
+            case (in, _) => in
+            }
+          checkExpectsInputType.withNewChildren(children)
+    }
+
+    def shouldCast(types: Seq[Expression]): Boolean = {
+      types.filter(e => hasStringType(e.dataType))
+        .map(e => extractStringType(e.dataType).collationId).distinct.size > 1
+    }
+
+    /**
+     * Whether the data type contains StringType.
+     */
+    @tailrec

Review Comment:
   You have `existsRecursively` which we use for similar purpose in QP side. E.g.
   ```scala
     def isBinaryStable(dataType: DataType): Boolean = dataType.existsRecursively {
       case st: StringType => CollationFactory.fetchCollation(st.collationId).isBinaryCollation
       case _ => true
     }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "dbatomic (via GitHub)" <gi...@apache.org>.
dbatomic commented on PR #45383:
URL: https://github.com/apache/spark/pull/45383#issuecomment-2007085232

   LGTM (alongside some comments). Btw, great work, this was a tough one + this PR will solve significant number of problems in collation space.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546046606


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression

Review Comment:
   Also, I do not believe incorporating string collations into core functions is the best way to go. If you take a look at findWiderCommonType, it is implemented differently for AnsiTypeCoercion and TypeCoercion. Firstly both of those take leftFold and go through expressions in order, but in our case that is not what we want. We want to first know what collations are explicit and then look for anything else. This is important because we want to have clear design on which error has to come first, and explicit mismatch should always come before implicit one. Another thing is that AnsiTypeCoercion seems to behave differently for StringTypes, we do not reorder any type in the sequence, which would result in implementing the same thing we did here with addition of another rule, but just in a core, already well defined function.
   One more thing I would add is that not all rules actually use core functions, e.g. ConcatCoercion. Also doing collations casting in one rule is way more efficient, as otherwise we would have to reorder StringTypes in every other rule, to make sure explicit collation mismatches are thrown first, which would result in multiple reorderings as opposed to constant of 2, one at the beginning for all StringTypes and the other at the end for all expressions that were cast to StringType from other types in some other implicit casting rules.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546398212


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   I'm not convinced. The propagation of "explicit collation" should be well-defined. Why don't funcitons like `substring`, `concat` propagate "explicit collation"?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1544060704


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala:
##########
@@ -112,12 +112,14 @@ case class Collate(child: Expression, collationName: String)
   since = "4.0.0",
   group = "string_funcs")
 // scalastyle:on line.contains.tab
-case class Collation(child: Expression) extends UnaryExpression with RuntimeReplaceable {
-  override def dataType: DataType = StringType
+case class Collation(child: Expression)
+  extends UnaryExpression with RuntimeReplaceable with ExpectsInputTypes {
+  override def dataType: DataType = child.dataType

Review Comment:
   shouldn't the collation name be a normal string type?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546114682


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression

Review Comment:
   I went through all the core functions and it seems like StringType implicit castings are already blocked, or go to default branch which is None. Is there something else we want to check?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "mihailom-db (via GitHub)" <gi...@apache.org>.
mihailom-db commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546388826


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>

Review Comment:
   There are only 2 expressions that are potentially missing. One is Elt, but I figured this one can be done when we actually do the support for this function. And the other one is Stack, but for this one I would say we do not want to do casting, as we actually pass columns there. Additionally, as a followup, I have a ticket to check whether we want to do Map casting of stringTypes and in which way, but this feels like a less of a priority to indeterminate collation. I could add support for Elt, it is a single line of code, but I thought it would be better to work on it in a single PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1546517397


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   We should play out how to the behavior.
   If I want combine arrays or maps, how do I do that. Is this what drives us towards CAST support. And does that imply that teh result of a CAST with a COLLATE clause (on an element/key/value) is a explicit collation)?  
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547191461


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)

Review Comment:
   the code looks confusing. If the default collation is not treated as implicit collation, we should filter it out here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547196013


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>

Review Comment:
   I disagree with special-case array type. The code looks broken. It assumes the children of the given expression can have both string type and array of string type, then tries to find a common collation between the string type child and the array element. This makes no sense without knowing the semantic of the given expression.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547390616


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, CreateArray, Elt, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+               | _: ComplexTypeMergingExpression
+               | _: CreateArray
+               | _: Elt
+               | _: ExpectsInputTypes
+               | _: Predicate
+               | _: SortOrder) =>
+      val newChildren = collateToSingleType(sc.children)

Review Comment:
   > If I understood correctly, we do not want to replicate the code from ImplicitTypeCasts.
   
   I don't think this is possible without significant refactoring. Correctness is more important than duplicated code at this stage.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547814185


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, Cast, Collate, ComplexTypeMergingExpression, ConcatWs, CreateArray, Expression, In, InSubquery, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+abstract class CollationTypeCasts extends TypeCoercionRule {
+  /**
+   * Extracts StringTypes from filtered hasStringType
+   */
+  @tailrec
+  private def extractStringType(dt: DataType): StringType = dt match {
+    case st: StringType => st
+    case ArrayType(et, _) => extractStringType(et)
+  }
+
+  /**
+   * Casts given expression to collated StringType with id equal to collationId only
+   * if expression has StringType in the first place.
+   * @param expr
+   * @param collationId
+   * @return
+   */
+  def castStringType(expr: Expression, st: StringType): Option[Expression] =
+    castStringType(expr.dataType, st).map { dt => Cast(expr, dt)}
+
+  private def castStringType(inType: AbstractDataType, castType: StringType): Option[DataType] = {
+    @Nullable val ret: DataType = inType match {
+      case st: StringType if st.collationId != castType.collationId => castType
+      case ArrayType(arrType, nullable) =>
+        castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull
+      case _ => null
+    }
+    Option(ret)
+  }
+
+  /**
+   * Collates input expressions to a single collation.
+   */
+  def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = {
+    val st = getOutputCollation(exprs)
+
+    exprs.map(e => castStringType(e, st).getOrElse(e))
+  }
+
+  /**
+   * Based on the data types of the input expressions this method determines
+   * a collation type which the output will have. This function accepts Seq of
+   * any expressions, but will only be affected by collated StringTypes or
+   * complex DataTypes with collated StringTypes (e.g. ArrayType)
+   */
+  def getOutputCollation(expr: Seq[Expression]): StringType = {
+    val explicitTypes = expr.filter(_.isInstanceOf[Collate])
+      .map(_.dataType.asInstanceOf[StringType].collationId)
+      .distinct
+
+    explicitTypes.size match {
+      // We have 1 explicit collation
+      case 1 => StringType(explicitTypes.head)
+      // Multiple explicit collations occurred
+      case size if size > 1 =>
+        throw QueryCompilationErrors
+          .explicitCollationMismatchError(
+            explicitTypes.map(t => StringType(t).typeName)
+          )
+      // Only implicit or default collations present
+      case 0 =>
+        val implicitTypes = expr.map(_.dataType)
+          .filter(hasStringType)
+          .map(extractStringType)
+          .filter(dt => dt.collationId != SQLConf.get.defaultStringType.collationId)
+
+        if (hasMultipleImplicits(implicitTypes)) {
+          throw QueryCompilationErrors.implicitCollationMismatchError()
+        }
+        else {
+          implicitTypes.find(dt => !(dt == SQLConf.get.defaultStringType))
+            .getOrElse(SQLConf.get.defaultStringType)
+        }
+    }
+  }
+
+  /**
+   * This check is always preformed when we have no explicit collation. It returns true
+   * if there are more than one implicit collations. Collations are distinguished by their
+   * collationId.
+   * @param dataTypes
+   * @return
+   */
+  private def hasMultipleImplicits(dataTypes: Seq[StringType]): Boolean =
+    dataTypes.map(_.collationId)
+      .filter(dt => !(dt == SQLConf.get.defaultStringType.collationId)).distinct.size > 1
+
+}
+
+/**
+ * This rule is used to collate all existing expressions related to StringType into a single
+ * collation. Arrays are handled using their elementType and should be cast for these expressions.
+ */
+object PreCollationTypeCasts extends CollationTypeCasts {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc@(_: In
+             | _: InSubquery
+             | _: CreateArray
+             | _: ComplexTypeMergingExpression
+             | _: ArrayJoin
+             | _: BinaryExpression
+             | _: ConcatWs
+             | _: Substring) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+  }
+}
+
+/**
+ *  This rule is used for managing expressions that have possible implicit casts from different
+ *  types in ImplicitTypeCasts rule.
+ */
+object PostCollationTypeCasts extends CollationTypeCasts {

Review Comment:
   Catalyst rules are executed with a loop, it's OK to wait for the next iteration and adjust the implicit casts.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1549239292


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType}
+import org.apache.spark.sql.catalyst.expressions.{ArrayJoin, BinaryExpression, CaseWhen, Cast, Coalesce, Collate, Concat, ConcatWs, CreateArray, Expression, Greatest, If, In, InSubquery, Least, Substring}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case ifExpr: If =>
+      ifExpr.withNewChildren(
+        ifExpr.predicate +: collateToSingleType(Seq(ifExpr.trueValue, ifExpr.falseValue)))
+    case caseWhenExpr: CaseWhen =>
+      val newValues = collateToSingleType(
+        caseWhenExpr.branches.map(b => b._2) ++ caseWhenExpr.elseValue)
+      caseWhenExpr.withNewChildren(
+        interleave(Seq.empty, caseWhenExpr.branches.map(b => b._1), newValues))
+    case substrExpr: Substring =>
+      // This case is necessary for changing Substring input to implicit collation
+      substrExpr.withNewChildren(
+        collateToSingleType(Seq(substrExpr.str)) :+ substrExpr.pos :+ substrExpr.len)

Review Comment:
   I don't get it. Why do we find the common collation for a single type?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1547984669


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCasts.scala:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.catalyst.analysis
+
+import javax.annotation.Nullable
+
+import scala.annotation.tailrec
+
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion.hasStringType
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Cast, ComplexTypeMergingExpression, CreateArray, ExpectsInputTypes, Expression, Predicate, SortOrder}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, StringType}
+
+object CollationTypeCasts extends TypeCoercionRule {
+  override val transform: PartialFunction[Expression, Expression] = {
+    case e if !e.childrenResolved => e
+    case sc @ (_: BinaryExpression
+              | _: Predicate
+              | _: SortOrder
+              | _: ExpectsInputTypes
+              | _: ComplexTypeMergingExpression) =>
+      val newChildren = collateToSingleType(sc.children)
+      sc.withNewChildren(newChildren)
+    case pesc @ (_: CreateArray) =>

Review Comment:
   Interesting. Yes array() sports an implicit collation. 'c' COLLATE UNICODE_CI sports an explicit collation.
   The rules are clear that the explicit collation wins.
   So the result is array<STRING UNICODE_CI> with an IMPLICIT collation.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL] Addition of implicit casting without indeterminate support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan closed pull request #45383: [SPARK-47210][SQL] Addition of implicit casting without indeterminate support
URL: https://github.com/apache/spark/pull/45383


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47210][SQL][COLLATION] Implicit casting on collated expressions [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #45383:
URL: https://github.com/apache/spark/pull/45383#discussion_r1537183989


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##########
@@ -182,16 +184,19 @@ object AnsiTypeCoercion extends TypeCoercionBase {
 
       // If a function expects a StringType, no StringType instance should be implicitly cast to
       // StringType with a collation that's not accepted (aka. lockdown unsupported collations).
-      case (_: StringType, StringType) => None
+      case (_: StringType, _: StringType) => None
       case (_: StringType, _: StringTypeCollated) => None
 
+      case (DateType, AnyTimestampType) =>

Review Comment:
   why do we need this change? It's not even related to string type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org