This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 08c6bb9bf32f [SPARK-47267][SQL] Add collation support for hash 
expressions
08c6bb9bf32f is described below

commit 08c6bb9bf32f31b5b9870d56cc4c16ab97616da6
Author: Uros Bojanic <157381213+uros...@users.noreply.github.com>
AuthorDate: Tue May 7 17:13:34 2024 +0800

    [SPARK-47267][SQL] Add collation support for hash expressions
    
    ### What changes were proposed in this pull request?
    Introduce collation awareness for hash expressions: MD5, SHA2, SHA1, CRC32, 
MURMUR3, XXHASH64.
    
    ### Why are the changes needed?
    Add collation support for hash expressions in Spark.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes, users should now be able to use collated strings within arguments for 
hash functions: md5, sha2, sha1, crc32, hash, xxhash64.
    
    ### How was this patch tested?
    E2e sql tests.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #46422 from uros-db/hash-expressions.
    
    Authored-by: Uros Bojanic <157381213+uros...@users.noreply.github.com>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../spark/sql/catalyst/expressions/hash.scala      |   6 +-
 .../spark/sql/CollationSQLExpressionsSuite.scala   | 179 +++++++++++++++++++++
 2 files changed, 182 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
index 5089cea136a8..fa342f641509 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
@@ -63,7 +63,7 @@ import org.apache.spark.util.ArrayImplicits._
 case class Md5(child: Expression)
   extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant {
 
-  override def dataType: DataType = StringType
+  override def dataType: DataType = SQLConf.get.defaultStringType
 
   override def inputTypes: Seq[DataType] = Seq(BinaryType)
 
@@ -103,7 +103,7 @@ case class Md5(child: Expression)
 case class Sha2(left: Expression, right: Expression)
   extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant 
with Serializable {
 
-  override def dataType: DataType = StringType
+  override def dataType: DataType = SQLConf.get.defaultStringType
   override def nullable: Boolean = true
 
   override def inputTypes: Seq[DataType] = Seq(BinaryType, IntegerType)
@@ -169,7 +169,7 @@ case class Sha2(left: Expression, right: Expression)
 case class Sha1(child: Expression)
   extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant {
 
-  override def dataType: DataType = StringType
+  override def dataType: DataType = SQLConf.get.defaultStringType
 
   override def inputTypes: Seq[DataType] = Seq(BinaryType)
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala
index fa82405109f1..596923d975a5 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala
@@ -28,6 +28,185 @@ class CollationSQLExpressionsSuite
   extends QueryTest
   with SharedSparkSession {
 
+  test("Support Md5 hash expression with collation") {
+    case class Md5TestCase(
+      input: String,
+      collationName: String,
+      result: String
+    )
+
+    val testCases = Seq(
+      Md5TestCase("Spark", "UTF8_BINARY", "8cde774d6f7333752ed72cacddb05126"),
+      Md5TestCase("Spark", "UTF8_BINARY_LCASE", 
"8cde774d6f7333752ed72cacddb05126"),
+      Md5TestCase("SQL", "UNICODE", "9778840a0100cb30c982876741b0b5a2"),
+      Md5TestCase("SQL", "UNICODE_CI", "9778840a0100cb30c982876741b0b5a2")
+    )
+
+    // Supported collations
+    testCases.foreach(t => {
+      val query =
+        s"""
+           |select md5('${t.input}')
+           |""".stripMargin
+      // Result & data type
+      withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) {
+        val testQuery = sql(query)
+        checkAnswer(testQuery, Row(t.result))
+        val dataType = StringType(t.collationName)
+        assert(testQuery.schema.fields.head.dataType.sameType(dataType))
+      }
+    })
+  }
+
+  test("Support Sha2 hash expression with collation") {
+    case class Sha2TestCase(
+      input: String,
+      collationName: String,
+      bitLength: Int,
+      result: String
+    )
+
+    val testCases = Seq(
+      Sha2TestCase("Spark", "UTF8_BINARY", 256,
+        "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"),
+      Sha2TestCase("Spark", "UTF8_BINARY_LCASE", 256,
+        "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"),
+      Sha2TestCase("SQL", "UNICODE", 256,
+        "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"),
+      Sha2TestCase("SQL", "UNICODE_CI", 256,
+        "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35")
+    )
+
+    // Supported collations
+    testCases.foreach(t => {
+      val query =
+        s"""
+           |select sha2('${t.input}', ${t.bitLength})
+           |""".stripMargin
+      // Result & data type
+      withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) {
+        val testQuery = sql(query)
+        checkAnswer(testQuery, Row(t.result))
+        val dataType = StringType(t.collationName)
+        assert(testQuery.schema.fields.head.dataType.sameType(dataType))
+      }
+    })
+  }
+
+  test("Support Sha1 hash expression with collation") {
+    case class Sha1TestCase(
+      input: String,
+      collationName: String,
+      result: String
+    )
+
+    val testCases = Seq(
+      Sha1TestCase("Spark", "UTF8_BINARY", 
"85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"),
+      Sha1TestCase("Spark", "UTF8_BINARY_LCASE", 
"85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"),
+      Sha1TestCase("SQL", "UNICODE", 
"2064cb643caa8d9e1de12eea7f3e143ca9f8680d"),
+      Sha1TestCase("SQL", "UNICODE_CI", 
"2064cb643caa8d9e1de12eea7f3e143ca9f8680d")
+    )
+
+    // Supported collations
+    testCases.foreach(t => {
+      val query =
+        s"""
+           |select sha1('${t.input}')
+           |""".stripMargin
+      // Result & data type
+      withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) {
+        val testQuery = sql(query)
+        checkAnswer(testQuery, Row(t.result))
+        val dataType = StringType(t.collationName)
+        assert(testQuery.schema.fields.head.dataType.sameType(dataType))
+      }
+    })
+  }
+
+  test("Support Crc32 hash expression with collation") {
+    case class Crc321TestCase(
+     input: String,
+     collationName: String,
+     result: Int
+    )
+
+    val testCases = Seq(
+      Crc321TestCase("Spark", "UTF8_BINARY", 1557323817),
+      Crc321TestCase("Spark", "UTF8_BINARY_LCASE", 1557323817),
+      Crc321TestCase("SQL", "UNICODE", 1299261525),
+      Crc321TestCase("SQL", "UNICODE_CI", 1299261525)
+    )
+
+    // Supported collations
+    testCases.foreach(t => {
+      val query =
+        s"""
+           |select crc32('${t.input}')
+           |""".stripMargin
+      // Result
+      withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) {
+        val testQuery = sql(query)
+        checkAnswer(testQuery, Row(t.result))
+      }
+    })
+  }
+
+  test("Support Murmur3Hash hash expression with collation") {
+    case class Murmur3HashTestCase(
+     input: String,
+     collationName: String,
+     result: Int
+    )
+
+    val testCases = Seq(
+      Murmur3HashTestCase("Spark", "UTF8_BINARY", 228093765),
+      Murmur3HashTestCase("Spark", "UTF8_BINARY_LCASE", 228093765),
+      Murmur3HashTestCase("SQL", "UNICODE", 17468742),
+      Murmur3HashTestCase("SQL", "UNICODE_CI", 17468742)
+    )
+
+    // Supported collations
+    testCases.foreach(t => {
+      val query =
+        s"""
+           |select hash('${t.input}')
+           |""".stripMargin
+      // Result
+      withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) {
+        val testQuery = sql(query)
+        checkAnswer(testQuery, Row(t.result))
+      }
+    })
+  }
+
+  test("Support XxHash64 hash expression with collation") {
+    case class XxHash64TestCase(
+      input: String,
+      collationName: String,
+      result: Long
+    )
+
+    val testCases = Seq(
+      XxHash64TestCase("Spark", "UTF8_BINARY", -4294468057691064905L),
+      XxHash64TestCase("Spark", "UTF8_BINARY_LCASE", -4294468057691064905L),
+      XxHash64TestCase("SQL", "UNICODE", -2147923034195946097L),
+      XxHash64TestCase("SQL", "UNICODE_CI", -2147923034195946097L)
+    )
+
+    // Supported collations
+    testCases.foreach(t => {
+      val query =
+        s"""
+           |select xxhash64('${t.input}')
+           |""".stripMargin
+      // Result
+      withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) {
+        val testQuery = sql(query)
+        checkAnswer(testQuery, Row(t.result))
+      }
+    })
+  }
+
   test("Conv expression with collation") {
     // Supported collations
     case class ConvTestCase(


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

Reply via email to