Skip to content

Commit eeb1566

Browse files
kazantsev-maksimKazantsev Maksimcomphead
authored
Feat: Add sha1 function impl (apache#2471)
* Add sha1 function impl * Fix format and tests * resolve conflicts * Apply suggestion from @comphead * Fix fallback message * Fix fallback message * Fix compile error --------- Co-authored-by: Kazantsev Maksim <[email protected]> Co-authored-by: Oleks V <[email protected]>
1 parent 88797d3 commit eeb1566

File tree

4 files changed

+23
-4
lines changed

4 files changed

+23
-4
lines changed

native/core/src/execution/jni_api.rs

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ use datafusion_comet_proto::spark_operator::Operator;
4343
use datafusion_spark::function::bitwise::bit_get::SparkBitGet;
4444
use datafusion_spark::function::datetime::date_add::SparkDateAdd;
4545
use datafusion_spark::function::datetime::date_sub::SparkDateSub;
46+
use datafusion_spark::function::hash::sha1::SparkSha1;
4647
use datafusion_spark::function::hash::sha2::SparkSha2;
4748
use datafusion_spark::function::math::expm1::SparkExpm1;
4849
use datafusion_spark::function::string::char::CharFunc;
@@ -332,6 +333,7 @@ fn prepare_datafusion_session_context(
332333
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitGet::default()));
333334
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateAdd::default()));
334335
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateSub::default()));
336+
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default()));
335337

336338
// Must be the last one to override existing functions with the same name
337339
datafusion_comet_spark_expr::register_all_comet_functions(&mut session_ctx)?;

spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -156,7 +156,8 @@ object QueryPlanSerde extends Logging with CometExprShim {
156156
classOf[Md5] -> CometScalarFunction("md5"),
157157
classOf[Murmur3Hash] -> CometMurmur3Hash,
158158
classOf[Sha2] -> CometSha2,
159-
classOf[XxHash64] -> CometXxHash64)
159+
classOf[XxHash64] -> CometXxHash64,
160+
classOf[Sha1] -> CometSha1)
160161

161162
private val stringExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map(
162163
classOf[Ascii] -> CometScalarFunction("ascii"),

spark/src/main/scala/org/apache/comet/serde/hash.scala

Lines changed: 15 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@
1919

2020
package org.apache.comet.serde
2121

22-
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Murmur3Hash, Sha2, XxHash64}
22+
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Murmur3Hash, Sha1, Sha2, XxHash64}
2323
import org.apache.spark.sql.types.{DecimalType, IntegerType, LongType, StringType}
2424

2525
import org.apache.comet.CometSparkSessionExtensions.withInfo
@@ -89,6 +89,20 @@ object CometSha2 extends CometExpressionSerde[Sha2] {
8989
}
9090
}
9191

92+
object CometSha1 extends CometExpressionSerde[Sha1] {
93+
override def convert(
94+
expr: Sha1,
95+
inputs: Seq[Attribute],
96+
binding: Boolean): Option[ExprOuterClass.Expr] = {
97+
if (!HashUtils.isSupportedType(expr)) {
98+
withInfo(expr, s"HashUtils doesn't support dataType: ${expr.child.dataType}")
99+
return None
100+
}
101+
val childExpr = exprToProtoInternal(expr.child, inputs, binding)
102+
scalarFunctionExprToProtoWithReturnType("sha1", StringType, false, childExpr)
103+
}
104+
}
105+
92106
private object HashUtils {
93107
def isSupportedType(expr: Expression): Boolean = {
94108
for (child <- expr.children) {

spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2024,7 +2024,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper {
20242024
|md5(col), md5(cast(a as string)), md5(cast(b as string)),
20252025
|hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col),
20262026
|xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col),
2027-
|sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1)
2027+
|sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1),
2028+
|sha1(col), sha1(cast(a as string)), sha1(cast(b as string))
20282029
|from test
20292030
|""".stripMargin)
20302031
}
@@ -2136,7 +2137,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper {
21362137
|md5(col), md5(cast(a as string)), --md5(cast(b as string)),
21372138
|hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col),
21382139
|xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col),
2139-
|sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1)
2140+
|sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1),
2141+
|sha1(col), sha1(cast(a as string))
21402142
|from test
21412143
|""".stripMargin)
21422144
}

0 commit comments

Comments
 (0)