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
1 change: 1 addition & 0 deletions docs/source/user-guide/latest/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,7 @@ These settings can be used to determine which parts of the plan are accelerated
| `spark.comet.expression.CheckOverflow.enabled` | Enable Comet acceleration for `CheckOverflow` | true |
| `spark.comet.expression.Chr.enabled` | Enable Comet acceleration for `Chr` | true |
| `spark.comet.expression.Coalesce.enabled` | Enable Comet acceleration for `Coalesce` | true |
| `spark.comet.expression.Concat.enabled` | Enable Comet acceleration for `Concat` | true |
| `spark.comet.expression.ConcatWs.enabled` | Enable Comet acceleration for `ConcatWs` | true |
| `spark.comet.expression.Contains.enabled` | Enable Comet acceleration for `Contains` | true |
| `spark.comet.expression.Cos.enabled` | Enable Comet acceleration for `Cos` | true |
Expand Down
4 changes: 0 additions & 4 deletions fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala
Original file line number Diff line number Diff line change
Expand Up @@ -126,15 +126,11 @@ object Meta {
SparkTypeOneOf(
Seq(
SparkStringType,
SparkNumericType,
SparkBinaryType,
SparkArrayType(
SparkTypeOneOf(Seq(SparkStringType, SparkNumericType, SparkBinaryType))))),
SparkTypeOneOf(
Seq(
SparkStringType,
SparkNumericType,
SparkBinaryType,
SparkArrayType(
SparkTypeOneOf(Seq(SparkStringType, SparkNumericType, SparkBinaryType))))))),
createFunctionWithInputTypes("concat_ws", Seq(SparkStringType, SparkStringType)),
Expand Down
16 changes: 10 additions & 6 deletions native/core/src/execution/jni_api.rs
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ use datafusion_spark::function::hash::sha1::SparkSha1;
use datafusion_spark::function::hash::sha2::SparkSha2;
use datafusion_spark::function::math::expm1::SparkExpm1;
use datafusion_spark::function::string::char::CharFunc;
use datafusion_spark::function::string::concat::SparkConcat;
use futures::poll;
use futures::stream::StreamExt;
use jni::objects::JByteBuffer;
Expand Down Expand Up @@ -317,20 +318,23 @@ fn prepare_datafusion_session_context(
let mut session_ctx = SessionContext::new_with_config_rt(session_config, Arc::new(runtime));

datafusion::functions_nested::register_all(&mut session_ctx)?;
register_datafusion_spark_function(&session_ctx);
// Must be the last one to override existing functions with the same name
datafusion_comet_spark_expr::register_all_comet_functions(&mut session_ctx)?;

Ok(session_ctx)
}

// register UDFs from datafusion-spark crate
// 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(SparkSha1::default()));

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

Ok(session_ctx)
session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default()));
}

/// Prepares arrow arrays for output.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,7 @@ object QueryPlanSerde extends Logging with CometExprShim {
classOf[BitLength] -> CometScalarFunction("bit_length"),
classOf[Chr] -> CometScalarFunction("char"),
classOf[ConcatWs] -> CometScalarFunction("concat_ws"),
classOf[Concat] -> CometConcat,
classOf[Contains] -> CometScalarFunction("contains"),
classOf[EndsWith] -> CometScalarFunction("ends_with"),
classOf[InitCap] -> CometInitCap,
Expand Down
14 changes: 13 additions & 1 deletion spark/src/main/scala/org/apache/comet/serde/strings.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ package org.apache.comet.serde

import java.util.Locale

import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Expression, InitCap, Length, Like, Literal, Lower, RegExpReplace, RLike, StringLPad, StringRepeat, StringRPad, Substring, Upper}
import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, Expression, InitCap, Length, Like, Literal, Lower, RegExpReplace, RLike, StringLPad, StringRepeat, StringRPad, Substring, Upper}
import org.apache.spark.sql.types.{BinaryType, DataTypes, LongType, StringType}

import org.apache.comet.CometConf
Expand Down Expand Up @@ -113,6 +113,18 @@ object CometSubstring extends CometExpressionSerde[Substring] {
}
}

object CometConcat extends CometScalarFunction[Concat]("concat") {
val unsupportedReason = "CONCAT supports only string input parameters"

override def getSupportLevel(expr: Concat): SupportLevel = {
if (expr.children.forall(_.dataType == DataTypes.StringType)) {
Compatible()
} else {
Incompatible(Some(unsupportedReason))
}
}
}

object CometLike extends CometExpressionSerde[Like] {

override def convert(expr: Like, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
Expand Down
Loading
Loading