[PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-07 Thread via GitHub


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

   ### What changes were proposed in this pull request?
   
   
   
   ### Why are the changes needed?
   Currently, all `StringType` arguments passed to built-in string functions in 
Spark SQL get treated as binary strings. This behaviour is incorrect for almost 
all collationIds except the default (0), and we should instead warn the user if 
they try to use an unsupported collation for the given function. Over time, we 
should implement the appropriate support for these (function, collation) pairs, 
but until then - we should have a way to fail unsupported statements in query 
analysis.
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, users will now get appropriate errors when they try to use an 
unsupported collation with a given string function.
   
   
   ### How was this patch tested?
   Tests in CollationSuite to check if these functions work for binary 
collations and throw exceptions for others.
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   Yes.


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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-07 Thread via GitHub


dbatomic commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1516510742


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationUtils.scala:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.expressions
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{DataType, StringType}
+
+object CollationUtils {

Review Comment:
   I usually recommend to stay away from *Utils naming, since the name is not 
descriptive.
   
   Can this file be called `CollationTypeConstraints`?



##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationUtils.scala:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.expressions
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{DataType, StringType}
+
+object CollationUtils {
+  def checkCollationCompatibility(
+   superCheck: => TypeCheckResult,

Review Comment:
   Why do you need superCheck here? Can you just bail out before calling this 
check if super check is failure?



##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationUtils.scala:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.expressions
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{DataType, StringType}
+
+object CollationUtils {
+  def checkCollationCompatibility(
+   superCheck: => TypeCheckResult,
+   collationId: Int,
+   rightDataType: DataType
+ ): TypeCheckResult = {
+val checkResult = superCheck
+if (checkResult.isFailure) return checkResult
+// Additional check needed for collation compatibility
+val rightCollationId: Int = 
rightDataType.asInstanceOf[St

Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-07 Thread via GitHub


HyukjinKwon commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1517022572


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationUtils.scala:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.expressions
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{DataType, StringType}
+
+object CollationUtils {
+  def checkCollationCompatibility(
+   superCheck: => TypeCheckResult,
+   collationId: Int,
+   rightDataType: DataType
+ ): TypeCheckResult = {
+val checkResult = superCheck
+if (checkResult.isFailure) return checkResult
+// Additional check needed for collation compatibility
+val rightCollationId: Int = 
rightDataType.asInstanceOf[StringType].collationId
+if (collationId != rightCollationId) {
+  return DataTypeMismatch(
+errorSubClass = "COLLATION_MISMATCH",
+messageParameters = Map(
+  "collationNameLeft" -> 
CollationFactory.fetchCollation(collationId).collationName,
+  "collationNameRight" -> 
CollationFactory.fetchCollation(rightCollationId).collationName
+)
+  )
+}
+TypeCheckResult.TypeCheckSuccess
+  }
+
+  final val SUPPORT_BINARY_ONLY: Int = 0
+  final val SUPPORT_LOWERCASE: Int = 1
+  final val SUPPORT_ALL_COLLATIONS: Int = 2
+
+  def checkCollationSupport(

Review Comment:
   Let's keep the indentation propery with 2/4 spaces 
(https://github.com/databricks/scala-style-guide)



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-07 Thread via GitHub


uros-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1517236308


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationUtils.scala:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.expressions
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{DataType, StringType}
+
+object CollationUtils {
+  def checkCollationCompatibility(
+   superCheck: => TypeCheckResult,

Review Comment:
   I could bail out for each function individually, but I think it's 
essentially the same thing? only this way, it's less copy-paste across the 
codebase



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-08 Thread via GitHub


MaxGekk commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1518490602


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationTypeConstraints.scala:
##
@@ -0,0 +1,89 @@
+/*
+ * 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.expressions
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{DataType, StringType}
+
+object CollationTypeConstraints {
+  def checkCollationCompatibility(
+  superCheck: => TypeCheckResult,
+  collationId: Int,
+  rightDataType: DataType): TypeCheckResult = {
+val checkResult = superCheck
+if (checkResult.isFailure) return checkResult
+// Additional check needed for collation compatibility
+val rightCollationId: Int = 
rightDataType.asInstanceOf[StringType].collationId
+if (collationId != rightCollationId) {
+  return DataTypeMismatch(
+errorSubClass = "COLLATION_MISMATCH",
+messageParameters = Map(
+  "collationNameLeft" -> 
CollationFactory.fetchCollation(collationId).collationName,
+  "collationNameRight" -> 
CollationFactory.fetchCollation(rightCollationId).collationName
+)
+  )
+}
+TypeCheckResult.TypeCheckSuccess
+  }
+
+  object CollationSupportLevel extends Enumeration {
+type CollationSupportLevel = Value
+
+val SUPPORT_BINARY_ONLY: Value = Value(0)
+val SUPPORT_LOWERCASE: Value = Value(1)
+val SUPPORT_ALL_COLLATIONS: Value = Value(2)
+  }
+
+  def checkCollationSupport(
+  superCheck: => TypeCheckResult,
+  collationId: Int,
+  functionName: String,
+  collationSupportLevel: CollationSupportLevel.CollationSupportLevel)
+  : TypeCheckResult = {
+val checkResult = superCheck
+if (checkResult.isFailure) return checkResult
+// Additional check needed for collation support
+val collation = CollationFactory.fetchCollation(collationId)
+collationSupportLevel match {
+  case CollationSupportLevel.SUPPORT_BINARY_ONLY =>
+if (!collation.isBinaryCollation) {
+  throwUnsupportedCollation(functionName, collation.collationName)
+}
+  case CollationSupportLevel.SUPPORT_LOWERCASE =>
+if (!collation.isBinaryCollation && !collation.isLowercaseCollation) {
+  throwUnsupportedCollation(functionName, collation.collationName)
+}
+  case CollationSupportLevel.SUPPORT_ALL_COLLATIONS => // No additional 
checks needed
+  case _ => throw new IllegalArgumentException("Invalid collation support 
level.")
+}
+TypeCheckResult.TypeCheckSuccess
+  }
+
+  private def throwUnsupportedCollation(functionName: String, collationName: 
String): Unit = {
+throw new SparkException(
+  errorClass = "UNSUPPORTED_COLLATION.FOR_FUNCTION",
+  messageParameters = Map(
+"functionName" -> functionName,

Review Comment:
   Please, quote it by `toSQLId`.



##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationTypeConstraints.scala:
##
@@ -0,0 +1,89 @@
+/*
+ * 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.expressions
+
+import org.apache.spark.SparkException
+import 

Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-12 Thread via GitHub


cloud-fan commented on PR #45422:
URL: https://github.com/apache/spark/pull/45422#issuecomment-1991557191

   Without updating `StringType.acceptsType`, I'm not confident to find out all 
functions that expect StringType but do not support 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-12 Thread via GitHub


uros-db commented on PR #45422:
URL: https://github.com/apache/spark/pull/45422#issuecomment-1991590743

   @cloud-fan yes, that is a problem... should we settle only on `string 
functions` for now? I think these functions that are meant to work with Strings 
are more sensitive to this error
   
   on a more important note, even if we were to update 
`StringType.acceptsType`, it would still not solve the problem of collation 
support level (introduced in this PR) that would prevent passing correctly 
matched arguments to a function that simply does not (yet) support that 
particular collation - we will be needing this much in the near future, and 
while we're at it - overriding `checkInputTypes` seems to solve both
   
   
   while type coercion is a separate effort, and will probably cover other 
parts of the codebase, what do we think about implementing this for now? 
@dbatomic
   
   > a bit more context for readers: for now, everything in the codebase that 
supports `StringType` will take `StringType(#)` (any collation) and treat it as 
the default collation (UTF8_BINARY); this is especially problematic for string 
functions that essentially return incorrect results without warning


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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-12 Thread via GitHub


cloud-fan commented on PR #45422:
URL: https://github.com/apache/spark/pull/45422#issuecomment-1991702090

   I don't think it's safe to only handle expressions in 
`regexpExpressions.scala`. For example, `Substring` is not there. I don't know 
how to collect all functions that take `StringType`, unless we check all 
expressions one by one.
   
   I'd prefer only updating functions that support collation to have more 
fine-grained collation check, which shouldn't be many right 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-13 Thread via GitHub


uros-db commented on PR #45422:
URL: https://github.com/apache/spark/pull/45422#issuecomment-1994394143

   @cloud-fan that makes a lot of sense, to combat this - now new case classes 
should handle this. essentially:
   - `StringType` no longer accepts all collationIds, but only the default 
collationId (0) - i.e. UTF8_BINARY
   - `StringTypeBinary` is added to allow binary collations (for now: 
UTF8_BINARY and UNICODE), but at this time we need this for full lockdown 
because casting is not ready (casting is a separate effort, and when it's done, 
we can have `StringType` accept all binary collations directly; for now, it's 
incorrect)
   - `StringTypeBinaryLcase` is added to allow binary & lowercase 
(UTF8_BINARY_LCASE, UTF8_BINARY, UNICODE) - this class is important because 
some expressions will support binary & lowercase, but not other collations at a 
given time
   - `StringTypeAllCollations` is added to allow all collations (for now this 
is supported only in StringPredicate expressions: Contains, StartsWith, 
EndsWith) - note that these expressions handle all collations, but can't 
guarantee that all string arguments have exactly the same collation type, so we 
still need `checkCollationCompatibility` in CollationTypeConstraints) once 
casting is ready, we will delete this


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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


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


##
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##
@@ -65,9 +64,43 @@ class StringType private(val collationId: Int) extends 
AtomicType with Serializa
 }
 
 /**
+ * Use StringType for expressions supporting only binary collation.
+ *
  * @since 1.3.0
  */
 @Stable
 case object StringType extends StringType(0) {
   def apply(collationId: Int): StringType = new StringType(collationId)
+  override private[sql] def acceptsType(other: DataType): Boolean =
+other.isInstanceOf[StringType] && 
other.asInstanceOf[StringType].isDefaultCollation

Review Comment:
   Is this needed? The default implementation should work well as it requires 
the collation id to be same as the current one, which is 0 (`case object 
StringType extends StringType(0)`). 



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -702,9 +702,13 @@ 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 {
+  case (expr: Expression, StringType) if 
expr.dataType.isInstanceOf[StringType] => expr
+  case (expr: Expression, StringTypeBinary | StringTypeBinaryLcase | 
StringTypeAnyCollation)
+if !expr.dataType.isInstanceOf[StringType] =>
+  implicitCast(expr, StringType).getOrElse(expr)

Review Comment:
   We should put the fix in `def implicitCast`. Otherwise it looks hacky to 
make a special case 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -956,9 +956,19 @@ object TypeCoercion extends TypeCoercionBase {
   })
   }
 
+  @tailrec
   override def implicitCast(e: Expression, expectedType: AbstractDataType): 
Option[Expression] = {
-implicitCast(e.dataType, expectedType).map { dt =>

Review Comment:
   Sorry I didn't make it clear. I mean the inner `implicitCast`, which has two 
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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


dbatomic commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1525990277


##
common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java:
##
@@ -69,6 +69,7 @@ public static class Collation {
  * byte for byte equal. All accent or case-insensitive collations are 
considered non-binary.
  */
 public final boolean isBinaryCollation;
+public final boolean isLowercaseCollation;

Review Comment:
   I think that the name is misleading. Also it doesn't make much sense to keep 
a flag that will be 0 for 99% of collations and 1 for only 1. You use this only 
in `StringType` so can you just say there:
   
   ```
   def isLcaseCollation: Boolean = collationId == LOWERCASE_COLLATION_ID
   ```
   That being said, maybe even we don't need this and you can just make this 
check in `StringTypeBinaryLcase`.
   
   I think that `isLowercaseCollation` is a very special edge case that 
shouldn't be baked in deeply in `Collation` object. `isBinary` is an actual 
property of collation and it makes sense to be here, but `isLowercaseCollation` 
is not.
   
   



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


dbatomic commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1525992996


##
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##
@@ -65,9 +64,41 @@ class StringType private(val collationId: Int) extends 
AtomicType with Serializa
 }
 
 /**
+ * Use StringType for expressions supporting only binary collation.
+ *
  * @since 1.3.0
  */
 @Stable
 case object StringType extends StringType(0) {
   def apply(collationId: Int): StringType = new StringType(collationId)
 }
+
+/**
+ * Use StringTypeBinary for expressions supporting only binary collation.
+ */
+case object StringTypeBinary extends AbstractDataType {

Review Comment:
   Does it make sense to move this into `CollationTypeConstraints.scala`?
   I would suggest to at least be explicit and say that these are not actual 
types that can be constructed/used externally.



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


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


##
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##
@@ -65,9 +64,41 @@ class StringType private(val collationId: Int) extends 
AtomicType with Serializa
 }
 
 /**
+ * Use StringType for expressions supporting only binary collation.
+ *
  * @since 1.3.0
  */
 @Stable
 case object StringType extends StringType(0) {
   def apply(collationId: Int): StringType = new StringType(collationId)
 }
+
+/**
+ * Use StringTypeBinary for expressions supporting only binary collation.
+ */
+case object StringTypeBinary extends AbstractDataType {

Review Comment:
   +1



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


dbatomic commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1526026385


##
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##
@@ -65,9 +64,41 @@ class StringType private(val collationId: Int) extends 
AtomicType with Serializa
 }
 
 /**
+ * Use StringType for expressions supporting only binary collation.
+ *
  * @since 1.3.0
  */
 @Stable
 case object StringType extends StringType(0) {
   def apply(collationId: Int): StringType = new StringType(collationId)
 }
+
+/**
+ * Use StringTypeBinary for expressions supporting only binary collation.
+ */
+case object StringTypeBinary extends AbstractDataType {
+  override private[sql] def defaultConcreteType: DataType = StringType
+  override private[sql] def simpleString: String = "string_binary"
+  override private[sql] def acceptsType(other: DataType): Boolean =
+other.isInstanceOf[StringType] && 
other.asInstanceOf[StringType].isBinaryCollation
+}
+
+/**
+ * Use StringTypeBinaryLcase for expressions supporting only binary and 
lowercase collation.
+ */
+case object StringTypeBinaryLcase extends AbstractDataType {
+  override private[sql] def defaultConcreteType: DataType = StringType
+  override private[sql] def simpleString: String = "string_binary_lcase"

Review Comment:
   Why do we need to provide `simpleString`? What is the situation where this 
is supposed to be exposed to 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


uros-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1526059294


##
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##
@@ -65,9 +64,41 @@ class StringType private(val collationId: Int) extends 
AtomicType with Serializa
 }
 
 /**
+ * Use StringType for expressions supporting only binary collation.
+ *
  * @since 1.3.0
  */
 @Stable
 case object StringType extends StringType(0) {
   def apply(collationId: Int): StringType = new StringType(collationId)
 }
+
+/**
+ * Use StringTypeBinary for expressions supporting only binary collation.
+ */
+case object StringTypeBinary extends AbstractDataType {
+  override private[sql] def defaultConcreteType: DataType = StringType
+  override private[sql] def simpleString: String = "string_binary"
+  override private[sql] def acceptsType(other: DataType): Boolean =
+other.isInstanceOf[StringType] && 
other.asInstanceOf[StringType].isBinaryCollation
+}
+
+/**
+ * Use StringTypeBinaryLcase for expressions supporting only binary and 
lowercase collation.
+ */
+case object StringTypeBinaryLcase extends AbstractDataType {
+  override private[sql] def defaultConcreteType: DataType = StringType
+  override private[sql] def simpleString: String = "string_binary_lcase"

Review Comment:
   `case object AbstractDataType` requires `simpleString`
   there are situations where this will be exposed
   
   for example, see that `case class Contains` (`StringPredicate`) has:
 override def inputTypes: Seq[AbstractDataType] =
   Seq(StringTypeAnyCollation, StringTypeAnyCollation)
   
   suppose someone passes something with some random type (not StringType), 
there will be a `DATATYPE_MISMATCH` exception saying:
   error, required: string_any_collation, passed: random type
   
   what we could do is keep the default parent implementation: `override 
private[sql] def simpleString: String = "string"` for StringTypeBinary, 
StringTypeBinaryLcase, StringTypeAnyCollation
   
   or keep separate simpleString overrides for these classes



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


uros-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1526059294


##
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##
@@ -65,9 +64,41 @@ class StringType private(val collationId: Int) extends 
AtomicType with Serializa
 }
 
 /**
+ * Use StringType for expressions supporting only binary collation.
+ *
  * @since 1.3.0
  */
 @Stable
 case object StringType extends StringType(0) {
   def apply(collationId: Int): StringType = new StringType(collationId)
 }
+
+/**
+ * Use StringTypeBinary for expressions supporting only binary collation.
+ */
+case object StringTypeBinary extends AbstractDataType {
+  override private[sql] def defaultConcreteType: DataType = StringType
+  override private[sql] def simpleString: String = "string_binary"
+  override private[sql] def acceptsType(other: DataType): Boolean =
+other.isInstanceOf[StringType] && 
other.asInstanceOf[StringType].isBinaryCollation
+}
+
+/**
+ * Use StringTypeBinaryLcase for expressions supporting only binary and 
lowercase collation.
+ */
+case object StringTypeBinaryLcase extends AbstractDataType {
+  override private[sql] def defaultConcreteType: DataType = StringType
+  override private[sql] def simpleString: String = "string_binary_lcase"

Review Comment:
   `case object AbstractDataType` requires `simpleString`
   there are situations where this will be exposed
   
   for example, see that `case class Contains` (`StringPredicate`) has:
 override def inputTypes: Seq[AbstractDataType] = 
Seq(StringTypeAnyCollation, StringTypeAnyCollation)
   
   suppose someone passes something with some random type (not StringType), 
there will be a `DATATYPE_MISMATCH` exception saying:
   error, required: string_any_collation, passed: random type
   
   what we could do is keep the default parent implementation: `override 
private[sql] def simpleString: String = "string"` for StringTypeBinary, 
StringTypeBinaryLcase, StringTypeAnyCollation
   
   or keep separate simpleString overrides for these classes



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,10 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
   // Cast any atomic type to string.
   case (any: AtomicType, StringType) if any != StringType => StringType

Review Comment:
   is it possible that a function uses `StringType(id)` as the expected input 
type? i.e. only allow string with certain collation 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##
@@ -205,6 +205,10 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   case (StringType, AnyTimestampType) =>
 Some(AnyTimestampType.defaultConcreteType)
 
+  // If the StringType has any collation other than UTF8_BINARY, it 
shouldn't be implicitly
+  // cast to StringType with collation 0.

Review Comment:
   looks the code does not follow what the comment says?



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


uros-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1526085933


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,10 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
   // Cast any atomic type to string.
   case (any: AtomicType, StringType) if any != StringType => StringType

Review Comment:
   I'm not aware that any functions exist that do this, it doesn't seem to make 
much sense either - functions should essentially only support 3 "groups" of 
collations (only binary, binary & lowercase, others)



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


uros-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1526092915


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##
@@ -205,6 +205,10 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   case (StringType, AnyTimestampType) =>
 Some(AnyTimestampType.defaultConcreteType)
 
+  // If the StringType has any collation other than UTF8_BINARY, it 
shouldn't be implicitly
+  // cast to StringType with collation 0.

Review Comment:
   perhaps the explanation is unclear, see how this sounds



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


uros-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1517236308


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationUtils.scala:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.expressions
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{DataType, StringType}
+
+object CollationUtils {
+  def checkCollationCompatibility(
+   superCheck: => TypeCheckResult,

Review Comment:
   edit: this is fixed 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


mihailom-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1526146584


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,10 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
   // Cast any atomic type to string.
   case (any: AtomicType, StringType) if any != StringType => StringType

Review Comment:
   Not that I know of as well. For now all functions get as inputType only 
StringType. This is the first PR that introduces something specific to some 
collations.



##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,10 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
   // Cast any atomic type to string.
   case (any: AtomicType, StringType) if any != StringType => StringType

Review Comment:
   +1 Not that I know of as well. For now all functions get as inputType only 
StringType. This is the first PR that introduces something specific to some 
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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


uros-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1526085933


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,10 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
   // Cast any atomic type to string.
   case (any: AtomicType, StringType) if any != StringType => StringType

Review Comment:
   I'm not aware that any functions exist that do this, it doesn't seem to make 
much sense either - functions should essentially only support 3 "groups" of 
collations (only binary, binary & lowercase, all 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


dbatomic commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1526193615


##
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##
@@ -40,6 +40,7 @@ class StringType private(val collationId: Int) extends 
AtomicType with Serializa
* equality and hashing).
*/
   def isBinaryCollation: Boolean = 
CollationFactory.fetchCollation(collationId).isBinaryCollation
+  def isLowercaseCollation: Boolean = collationId == 
CollationFactory.LOWERCASE_COLLATION_ID

Review Comment:
   Can you remove even this guy and push the check into `StringTypeBinaryLcase`?



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


dbatomic commented on PR #45422:
URL: https://github.com/apache/spark/pull/45422#issuecomment-1999648400

   LGTM
   
   As a follow up we should revisit error messages. IMO it is weird to expose 
message with "string_any_collation" type to customer. But I think that we can 
do that as a follow up.


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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-15 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,11 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
+  case (st: StringType, _: StringTypeCollated) => st
   // Cast any atomic type to string.
   case (any: AtomicType, StringType) if any != StringType => StringType
+  case (any: AtomicType, _: StringTypeCollated) if any != StringType => 
StringType

Review Comment:
   The code can be more readable if we call 
`StringTypeCollated#defaultConcreteType`, which is `StringType(0)`
   



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


mihailom-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1529966896


##
sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala:
##
@@ -40,6 +40,7 @@ class StringType private(val collationId: Int) extends 
AtomicType with Serializa
* equality and hashing).
*/
   def isBinaryCollation: Boolean = 
CollationFactory.fetchCollation(collationId).isBinaryCollation
+  def isLowercaseCollation: Boolean = collationId == 
CollationFactory.LOWERCASE_COLLATION_ID

Review Comment:
   I am not sure this is possible. StringTypeBinaryLcase does not extend 
StringType, and the point of this function for now is to call it on StringType 
object in acceptsType to check if we should let the function proceed with that 
input.



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


mihailom-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1530084248


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,11 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
+  case (st: StringType, _: StringTypeCollated) => st
   // Cast any atomic type to string.
   case (any: AtomicType, StringType) if any != StringType => StringType
+  case (any: AtomicType, _: StringTypeCollated) if any != StringType => 
StringType

Review Comment:
   Is this what you meant?



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##
@@ -205,6 +205,11 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   case (StringType, AnyTimestampType) =>
 Some(AnyTimestampType.defaultConcreteType)
 
+  // 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

Review Comment:
   isn't this case match covered by the first case match `case _ if 
expectedType.acceptsType(inType) => Some(inType)`?



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##
@@ -205,6 +205,11 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   case (StringType, AnyTimestampType) =>
 Some(AnyTimestampType.defaultConcreteType)
 
+  // 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

Review Comment:
   I think this should be `case (_: StringType, 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##
@@ -215,6 +220,10 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   None
 }
 
+  // "canANSIStoreAssign" doesn't account for targets extending 
StringTypeCollated, but
+  // ANSIStoreAssign is generally expected to return "true" for 
(AtomicType, StringType)
+  case (_: AtomicType, _: StringTypeCollated) => Some(StringType)

Review Comment:
   is this correct? `StringType` does not satisfy `StringTypeCollated`



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,12 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st

Review Comment:
   reading the code around here, I think `null` means no 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##
@@ -205,6 +205,11 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   case (StringType, AnyTimestampType) =>
 Some(AnyTimestampType.defaultConcreteType)
 
+  // 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, _: StringTypeCollated) => None

Review Comment:
   This case should be put before `case (StringType, a: AtomicType) =>`, 
otherwise it's useless



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,12 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
+  case (st: StringType, _: StringTypeCollated) => st

Review Comment:
   I think this will be covered by the last default case match?



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


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


##
sql/core/src/test/scala/org/apache/spark/sql/CollationRegexpExpressionsSuite.scala:
##
@@ -0,0 +1,438 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import scala.collection.immutable.Seq
+
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.test.SharedSparkSession
+
+class CollationRegexpExpressionsSuite extends QueryTest with 
SharedSparkSession {

Review Comment:
   can we add a `CollationRegexpExpressionsANSISuite extends 
CollationRegexpExpressionsSuite` to turn on ANSI mode?



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


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


##
sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala:
##
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import scala.collection.immutable.Seq
+
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.test.SharedSparkSession
+
+class CollationStringExpressionsSuite extends QueryTest with 
SharedSparkSession {

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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


mihailom-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1530992551


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala:
##
@@ -215,6 +220,10 @@ object AnsiTypeCoercion extends TypeCoercionBase {
   None
 }
 
+  // "canANSIStoreAssign" doesn't account for targets extending 
StringTypeCollated, but
+  // ANSIStoreAssign is generally expected to return "true" for 
(AtomicType, StringType)
+  case (_: AtomicType, _: StringTypeCollated) => Some(StringType)

Review Comment:
   Yes it is. canANSIStoreAssign has a rule for casting AtomicType to 
StringType, but since StringTypeCollated does not extend StringType, but only 
AbstractDataType, this cast rule will not be picked up. But I would say this 
rule has to be improved to check for all canANsiStoreAssign 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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


mihailom-db commented on code in PR #45422:
URL: https://github.com/apache/spark/pull/45422#discussion_r1531040306


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:
##
@@ -994,8 +994,12 @@ object TypeCoercion extends TypeCoercionBase {
   case (StringType, datetime: DatetimeType) => datetime
   case (StringType, AnyTimestampType) => 
AnyTimestampType.defaultConcreteType
   case (StringType, BinaryType) => BinaryType
+  case (st: StringType, StringType) => st
+  case (st: StringType, _: StringTypeCollated) => st

Review Comment:
   Yes and no. The following two lines would have made a cast, but I changed 
them so they doesn't.



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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-19 Thread via GitHub


mihailom-db commented on PR #45422:
URL: https://github.com/apache/spark/pull/45422#issuecomment-2008104827

   > LGTM
   > 
   > As a follow up we should revisit error messages. IMO it is weird to expose 
message with "string_any_collation" type to customer. But I think that we can 
do that as a follow up.
   
   Could this be an onboarding task?


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

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

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-20 Thread via GitHub


cloud-fan commented on PR #45422:
URL: https://github.com/apache/spark/pull/45422#issuecomment-2009773392

   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-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47296][SQL][COLLATION] Fail unsupported functions for non-binary collations [spark]

2024-03-20 Thread via GitHub


cloud-fan closed pull request #45422: [SPARK-47296][SQL][COLLATION] Fail 
unsupported functions for non-binary collations
URL: https://github.com/apache/spark/pull/45422


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

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

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


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