Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions native/Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 1 addition & 1 deletion native/core/src/execution/jni_api.rs
Original file line number Diff line number Diff line change
Expand Up @@ -393,14 +393,14 @@ fn prepare_datafusion_session_context(
// register UDFs from datafusion-spark crate
fn register_datafusion_spark_function(session_ctx: &SessionContext) {
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkExpm1::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha2::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(CharFunc::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitGet::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateAdd::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateSub::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkLastDay::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkNextDay::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha2::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitwiseNot::default()));
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkHex::default()));
Expand Down
8 changes: 7 additions & 1 deletion spark/src/main/scala/org/apache/comet/serde/hash.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

package org.apache.comet.serde

import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Murmur3Hash, Sha1, Sha2, XxHash64}
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Literal, Murmur3Hash, Sha1, Sha2, XxHash64}
import org.apache.spark.sql.types.{ArrayType, DataType, DecimalType, IntegerType, LongType, MapType, StringType, StructType}

import org.apache.comet.CometSparkSessionExtensions.withInfo
Expand Down Expand Up @@ -76,6 +76,12 @@ object CometSha2 extends CometExpressionSerde[Sha2] {
return None
}

// Fall back to Spark for literal input to avoid native engine crash (#3340)
if (expr.left.isInstanceOf[Literal]) {
withInfo(expr, "Sha2 with literal input falls back to Spark")
return None
}

// It's possible for spark to dynamically compute the number of bits from input
// expression, however DataFusion does not support that yet.
if (!expr.right.foldable) {
Expand Down
6 changes: 4 additions & 2 deletions spark/src/test/resources/sql-tests/expressions/hash/hash.sql
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,10 @@ statement
INSERT INTO test VALUES ('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999), ('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999)

query
SELECT md5(col), md5(cast(a as string)), md5(cast(b as string)), hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col), sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1), sha1(col), sha1(cast(a as string)), sha1(cast(b as string)) FROM test
SELECT md5(col), md5(cast(a as string)), md5(cast(b as string)), hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col), sha1(col), sha1(cast(a as string)), sha1(cast(b as string)), sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1) FROM test

-- literal arguments
query ignore(https://github.com/apache/datafusion-comet/issues/3340)
-- sha2 with literal input falls back to Spark to avoid native engine crash (#3340)
query expect_fallback(Sha2 with literal input falls back to Spark)
SELECT md5('Spark SQL'), sha1('test'), sha2('test', 256), hash('test'), xxhash64('test')

Original file line number Diff line number Diff line change
Expand Up @@ -1891,8 +1891,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper {
|hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col),
|xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col),
|crc32(col), crc32(cast(a as string)), crc32(cast(b as string)),
|sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1),
|sha1(col), sha1(cast(a as string)), sha1(cast(b as string))
|sha1(col), sha1(cast(a as string)), sha1(cast(b as string)),
|sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1)
|from test
|""".stripMargin)
}
Expand Down Expand Up @@ -2002,8 +2002,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper {
|hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col),
|xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col),
|crc32(col), crc32(cast(a as string)),
|sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1),
|sha1(col), sha1(cast(a as string))
|sha1(col), sha1(cast(a as string)),
|sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1)
|from test
|""".stripMargin)
}
Expand Down
Loading