Re: [PR] [SPARK-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteCollationJoin.scala:
##
@@ -0,0 +1,102 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
CollationKey, EqualNullSafe, EqualTo, Lower}
+import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{ArrayType, StringType, StructType}
+
+object RewriteCollationJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput {
+case j @ Join(_, _, _, Some(condition), _) =>
+  val newCondition = condition transform {
+case EqualTo(l: AttributeReference, r: AttributeReference) =>
+  (l.dataType, r.dataType) match {
+case (_: StringType, _: StringType) =>
+  val collationId = l.dataType.asInstanceOf[StringType].collationId
+  val collation = CollationFactory.fetchCollation(collationId)
+  if (collation.supportsBinaryEquality) {
+return plan

Review Comment:
   Isn't `plan` here only at the "Join" level, i.e. this should return the 
original "Join" as it was, without modification?
   
   if there is another join, it should be a completely separate visit to this 
rule (which is why now no tests are failing with multiple Joins, such as 
numerous TPCDS queries with collation) - which I also confirmed through 
debugging
   
   if I remove `return plan` here, various tests start failing, so I would say 
this should be correct with respect to multiple join conditions



-- 
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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteCollationJoin.scala:
##
@@ -0,0 +1,102 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
CollationKey, EqualNullSafe, EqualTo, Lower}
+import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{ArrayType, StringType, StructType}
+
+object RewriteCollationJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput {
+case j @ Join(_, _, _, Some(condition), _) =>
+  val newCondition = condition transform {
+case EqualTo(l: AttributeReference, r: AttributeReference) =>
+  (l.dataType, r.dataType) match {
+case (_: StringType, _: StringType) =>
+  val collationId = l.dataType.asInstanceOf[StringType].collationId
+  val collation = CollationFactory.fetchCollation(collationId)
+  if (collation.supportsBinaryEquality) {
+return plan

Review Comment:
   Isn't `plan` here only at the "Join" level, i.e. this should return the 
original "Join" as it was, without modification?
   
   if there is another join, it should be a completely separate visit to this 
rule (which is why now no tests are failing with multiple Joins, such as 
numerous TPCDS queries with collation)
   
   if I remove `return plan` here, various tests start failing, so I would say 
this should be correct with respect to multiple join conditions



-- 
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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteCollationJoin.scala:
##
@@ -0,0 +1,102 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
CollationKey, EqualNullSafe, EqualTo, Lower}
+import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{ArrayType, StringType, StructType}
+
+object RewriteCollationJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput {
+case j @ Join(_, _, _, Some(condition), _) =>
+  val newCondition = condition transform {
+case EqualTo(l: AttributeReference, r: AttributeReference) =>
+  (l.dataType, r.dataType) match {
+case (_: StringType, _: StringType) =>
+  val collationId = l.dataType.asInstanceOf[StringType].collationId
+  val collation = CollationFactory.fetchCollation(collationId)
+  if (collation.supportsBinaryEquality) {
+return plan

Review Comment:
   Isn't `plan` here only at the "Join" level, i.e. this should return the 
original "Join" as it was, without modification
   
   if there is another join, it should be a completely separate visit to this 
rule (which is why now no tests are failing with multiple Joins, such as 
numerous TPCDS queries with 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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala:
##
@@ -759,34 +761,38 @@ class CollationSuite extends DatasourceV2SQLBase with 
AdaptiveSparkPlanHelper {
   }
 
   test("hash based joins not allowed for non-binary collated strings") {
-val in = (('a' to 'z') ++ ('A' to 'Z')).map(_.toString * 3).map(e => 
Row.apply(e, e))
+spark.conf.set("spark.sql.codegen.wholeStage", false)
+val in = (('a' to 'b') ++ ('A' to 'B')).map(_.toString).map(e => 
Row.apply(e, e))

Review Comment:
   I assume that these are leftovers from some prototyping :)



-- 
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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala:
##
@@ -247,44 +247,36 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
   _, left, right, hint) =>
 val hashJoinSupport = hashJoinSupported(leftKeys, rightKeys)

Review Comment:
   I think that you still need to keep hashJoinSupport flag check. E.g. if 
physical plan is manually constructed you don't have any guarantee that 
CollationKey QO rule kicked in.
   
   I would prefer collation key returning binary normalized value that does 
support hash join and keep hashJoinSupport check for StringType.
   
   Also, in future, we may want to add rules in QO that will chose to avoid 
collation key path, given that that path is also not cheap and not clearly 
better than sort merge join (we need to measure 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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationKey.scala:
##
@@ -0,0 +1,161 @@
+/*
+ * 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 java.util.Locale
+
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.catalyst.util.{CollationFactory, GenericArrayData}
+import org.apache.spark.sql.internal.types.{AbstractArrayType, 
StringTypeAnyCollation}
+import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType, 
StringType, StructType, TypeCollection}
+import org.apache.spark.unsafe.types.UTF8String
+
+case class CollationKey(expr: Expression) extends UnaryExpression with 
ExpectsInputTypes {

Review Comment:
   So, this is saying that `CollationKey` for Array(string) will return 
Array(string), but string data will be garbage for ICU collations - it will 
just be constructed from binary representation of collation key, which may not 
even be valid UTF8 codes. Also, that "garbage" data will still go through 
comparison rules for string of given collation (instead it should be just 
binary comparison).
   
   Is it cleaner to just say that CollationKey is a StringType -> Binary 
transformation? Then other layers can take care of array and struct 
transformations? Also second part can be a separate 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-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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationKey.scala:
##
@@ -0,0 +1,161 @@
+/*
+ * 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 java.util.Locale

Review Comment:
   Let's avoid using JVM's locale. We will anyhow probably completely switch to 
ICU even for UTF8_BINARY_LCASE.
   
   If I am reading code correctly you don't even need Locale here. Just use 
`str.toLowerCase` instead of `str.toString.toLowerCase(Locale.ROOT)` 



-- 
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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationKey.scala:
##
@@ -0,0 +1,161 @@
+/*
+ * 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 java.util.Locale

Review Comment:
   We should not use JVM's Locale. Please use ICU's 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-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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CollationKey.scala:
##
@@ -0,0 +1,161 @@
+/*
+ * 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 java.util.Locale

Review Comment:
   We should not use JVM's Locale. Please use ICU's 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-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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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

   Can you add a bit more detailed comment in PR description about actual 
implementation?


-- 
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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteCollationJoin.scala:
##
@@ -0,0 +1,102 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
CollationKey, EqualNullSafe, EqualTo, Lower}
+import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{ArrayType, StringType, StructType}
+
+object RewriteCollationJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput {
+case j @ Join(_, _, _, Some(condition), _) =>
+  val newCondition = condition transform {
+case EqualTo(l: AttributeReference, r: AttributeReference) =>
+  (l.dataType, r.dataType) match {
+case (_: StringType, _: StringType) =>
+  val collationId = l.dataType.asInstanceOf[StringType].collationId
+  val collation = CollationFactory.fetchCollation(collationId)
+  if (collation.supportsBinaryEquality) {
+return plan
+  } else if (collation.supportsLowercaseEquality) {
+EqualTo(Lower(l), Lower(r))
+  } else {
+EqualTo(CollationKey(l), CollationKey(r))
+  }
+case (ArrayType(_: StringType, _), ArrayType(_: StringType, _)) =>

Review Comment:
   how about array of array of string? We should use recursion and make the 
code more general. Please follow `CharVarcharUtils`



-- 
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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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


##
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteCollationJoin.scala:
##
@@ -0,0 +1,102 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
CollationKey, EqualNullSafe, EqualTo, Lower}
+import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.CollationFactory
+import org.apache.spark.sql.types.{ArrayType, StringType, StructType}
+
+object RewriteCollationJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput {
+case j @ Join(_, _, _, Some(condition), _) =>
+  val newCondition = condition transform {
+case EqualTo(l: AttributeReference, r: AttributeReference) =>
+  (l.dataType, r.dataType) match {
+case (_: StringType, _: StringType) =>
+  val collationId = l.dataType.asInstanceOf[StringType].collationId
+  val collation = CollationFactory.fetchCollation(collationId)
+  if (collation.supportsBinaryEquality) {
+return plan

Review Comment:
   no, we shouldn't `return` within the transform lambda. The plan may have 
multiple joins and the `return` here reverts all rewrites.



-- 
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-48000][SQL] Enable hash join support for non-binary collations [spark]

2024-05-07 Thread via GitHub


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

   this is ready for review, @dbatomic @cloud-fan please provide some feedback 
on this approach
   also adding the rest of Belgrade SQL team: @mihailom-db @nikolamand-db 
@stefankandic @stevomitric


-- 
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