From 24427e04b300f71bd259d5c319e88fe34d3165ff Mon Sep 17 00:00:00 2001 From: noroshi <253434427+n0r0shi@users.noreply.github.com> Date: Tue, 24 Feb 2026 02:58:10 +0000 Subject: [PATCH] feat: support ilike expression Register datafusion-spark's SparkILike UDF and add CometILike handler in stringExpressions. Custom escape characters fall back to Spark. --- native/core/src/execution/jni_api.rs | 2 ++ .../apache/comet/serde/QueryPlanSerde.scala | 1 + .../org/apache/comet/serde/strings.scala | 15 ++++++++++- .../comet/CometStringExpressionSuite.scala | 25 +++++++++++++++++++ 4 files changed, 42 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 0193f3012c..65c674e3a8 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -55,6 +55,7 @@ use datafusion_spark::function::math::hex::SparkHex; use datafusion_spark::function::math::width_bucket::SparkWidthBucket; use datafusion_spark::function::string::char::CharFunc; use datafusion_spark::function::string::concat::SparkConcat; +use datafusion_spark::function::string::ilike::SparkILike; use futures::poll; use futures::stream::StreamExt; use jni::objects::JByteBuffer; @@ -400,6 +401,7 @@ fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkWidthBucket::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(MapFromEntries::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkCrc32::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkILike::default())); } /// Prepares arrow arrays for output. diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 9d13ccd9ed..a589c0c1eb 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -154,6 +154,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[Concat] -> CometConcat, classOf[Contains] -> CometScalarFunction("contains"), classOf[EndsWith] -> CometScalarFunction("ends_with"), + classOf[ILike] -> CometILike, classOf[InitCap] -> CometInitCap, classOf[Length] -> CometLength, classOf[Like] -> CometLike, diff --git a/spark/src/main/scala/org/apache/comet/serde/strings.scala b/spark/src/main/scala/org/apache/comet/serde/strings.scala index 64ba644048..9bf4ed1119 100644 --- a/spark/src/main/scala/org/apache/comet/serde/strings.scala +++ b/spark/src/main/scala/org/apache/comet/serde/strings.scala @@ -21,7 +21,7 @@ package org.apache.comet.serde import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, ConcatWs, Expression, If, InitCap, IsNull, Left, Length, Like, Literal, Lower, RegExpReplace, Right, RLike, StringLPad, StringRepeat, StringRPad, StringSplit, Substring, Upper} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, ConcatWs, Expression, If, ILike, InitCap, IsNull, Left, Length, Like, Literal, Lower, RegExpReplace, Right, RLike, StringLPad, StringRepeat, StringRPad, StringSplit, Substring, Upper} import org.apache.spark.sql.types.{BinaryType, DataTypes, LongType, StringType} import org.apache.spark.unsafe.types.UTF8String @@ -238,6 +238,19 @@ object CometLike extends CometExpressionSerde[Like] { } } +object CometILike extends CometExpressionSerde[ILike] { + + override def convert(expr: ILike, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = { + if (expr.escapeChar != '\\') { + withInfo(expr, s"custom escape character ${expr.escapeChar} not supported in ILIKE") + return None + } + val childExpr = expr.children.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = scalarFunctionExprToProto("ilike", childExpr: _*) + optExprWithInfo(optExpr, expr, expr.children: _*) + } +} + object CometRLike extends CometExpressionSerde[RLike] { override def convert(expr: RLike, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = { diff --git a/spark/src/test/scala/org/apache/comet/CometStringExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometStringExpressionSuite.scala index 121d7f7d5a..67170cb55c 100644 --- a/spark/src/test/scala/org/apache/comet/CometStringExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometStringExpressionSuite.scala @@ -148,6 +148,31 @@ class CometStringExpressionSuite extends CometTestBase { } } + test("ilike") { + withSQLConf("spark.comet.caseConversion.enabled" -> "true") { + withParquetTable(Seq("Hello", "WORLD", "hello", "FooBar", null).map(Tuple1(_)), "tbl") { + checkSparkAnswerAndOperator("SELECT _1 ILIKE '%hello%' FROM tbl") + checkSparkAnswerAndOperator("SELECT _1 ILIKE 'H_llo' FROM tbl") + checkSparkAnswerAndOperator("SELECT _1 ILIKE '%WORLD%' FROM tbl") + checkSparkAnswerAndOperator("SELECT NULL ILIKE '%test%' FROM tbl") + } + } + } + + test("ilike falls back when caseConversion is disabled") { + // ILIKE requires case-insensitive comparison which uses locale-specific + // case conversion (upper/lower). Rust's to_lowercase() follows Unicode + // default rules while Java uses locale-specific rules (e.g. Turkish I), + // so Comet falls back when caseConversion is disabled. + withSQLConf("spark.comet.caseConversion.enabled" -> "false") { + withParquetTable(Seq("Hello", "WORLD").map(Tuple1(_)), "tbl") { + checkSparkAnswerAndFallbackReason( + "SELECT _1 ILIKE '%hello%' FROM tbl", + "Comet is not compatible with Spark for case conversion") + } + } + } + test("split string basic") { withSQLConf("spark.comet.expression.StringSplit.allowIncompatible" -> "true") { withParquetTable((0 until 5).map(i => (s"value$i,test$i", i)), "tbl") {