Skip to content

Commit 035aeff

Browse files
authored
feat: Support left expression (apache#3206)
1 parent 48776fe commit 035aeff

File tree

4 files changed

+79
-1
lines changed

4 files changed

+79
-1
lines changed

docs/source/user-guide/latest/configs.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -269,6 +269,7 @@ These settings can be used to determine which parts of the plan are accelerated
269269
| `spark.comet.expression.JsonToStructs.enabled` | Enable Comet acceleration for `JsonToStructs` | true |
270270
| `spark.comet.expression.KnownFloatingPointNormalized.enabled` | Enable Comet acceleration for `KnownFloatingPointNormalized` | true |
271271
| `spark.comet.expression.LastDay.enabled` | Enable Comet acceleration for `LastDay` | true |
272+
| `spark.comet.expression.Left.enabled` | Enable Comet acceleration for `Left` | true |
272273
| `spark.comet.expression.Length.enabled` | Enable Comet acceleration for `Length` | true |
273274
| `spark.comet.expression.LessThan.enabled` | Enable Comet acceleration for `LessThan` | true |
274275
| `spark.comet.expression.LessThanOrEqual.enabled` | Enable Comet acceleration for `LessThanOrEqual` | true |

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -170,6 +170,7 @@ object QueryPlanSerde extends Logging with CometExprShim {
170170
classOf[StringTrimBoth] -> CometScalarFunction("btrim"),
171171
classOf[StringTrimLeft] -> CometScalarFunction("ltrim"),
172172
classOf[StringTrimRight] -> CometScalarFunction("rtrim"),
173+
classOf[Left] -> CometLeft,
173174
classOf[Substring] -> CometSubstring,
174175
classOf[Upper] -> CometUpper)
175176

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

Lines changed: 31 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ package org.apache.comet.serde
2121

2222
import java.util.Locale
2323

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

2727
import org.apache.comet.CometConf
@@ -113,6 +113,36 @@ object CometSubstring extends CometExpressionSerde[Substring] {
113113
}
114114
}
115115

116+
object CometLeft extends CometExpressionSerde[Left] {
117+
118+
override def convert(expr: Left, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
119+
expr.len match {
120+
case Literal(lenValue, _) =>
121+
exprToProtoInternal(expr.str, inputs, binding) match {
122+
case Some(strExpr) =>
123+
val builder = ExprOuterClass.Substring.newBuilder()
124+
builder.setChild(strExpr)
125+
builder.setStart(1)
126+
builder.setLen(lenValue.asInstanceOf[Int])
127+
Some(ExprOuterClass.Expr.newBuilder().setSubstring(builder).build())
128+
case None =>
129+
withInfo(expr, expr.str)
130+
None
131+
}
132+
case _ =>
133+
withInfo(expr, "LEFT len must be a literal")
134+
None
135+
}
136+
}
137+
138+
override def getSupportLevel(expr: Left): SupportLevel = {
139+
expr.str.dataType match {
140+
case _: BinaryType | _: StringType => Compatible()
141+
case _ => Unsupported(Some(s"LEFT does not support ${expr.str.dataType}"))
142+
}
143+
}
144+
}
145+
116146
object CometConcat extends CometScalarFunction[Concat]("concat") {
117147
val unsupportedReason = "CONCAT supports only string input parameters"
118148

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

Lines changed: 46 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -506,6 +506,52 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper {
506506
}
507507
}
508508

509+
test("LEFT function") {
510+
withParquetTable((0 until 10).map(i => (s"test$i", i)), "tbl") {
511+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, 2) FROM tbl")
512+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, 4) FROM tbl")
513+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, 0) FROM tbl")
514+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, -1) FROM tbl")
515+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, 100) FROM tbl")
516+
checkSparkAnswerAndOperator("SELECT LEFT(CAST(NULL AS STRING), 2) FROM tbl LIMIT 1")
517+
}
518+
}
519+
520+
test("LEFT function with unicode") {
521+
val data = Seq("café", "hello世界", "😀emoji", "తెలుగు")
522+
withParquetTable(data.zipWithIndex, "unicode_tbl") {
523+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, 2) FROM unicode_tbl")
524+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, 3) FROM unicode_tbl")
525+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, 0) FROM unicode_tbl")
526+
}
527+
}
528+
529+
test("LEFT function equivalence with SUBSTRING") {
530+
withParquetTable((0 until 20).map(i => Tuple1(s"test$i")), "equiv_tbl") {
531+
val df = spark.sql("""
532+
SELECT _1,
533+
LEFT(_1, 3) as left_result,
534+
SUBSTRING(_1, 1, 3) as substring_result
535+
FROM equiv_tbl
536+
""")
537+
checkAnswer(
538+
df.filter(
539+
"left_result != substring_result OR " +
540+
"(left_result IS NULL AND substring_result IS NOT NULL) OR " +
541+
"(left_result IS NOT NULL AND substring_result IS NULL)"),
542+
Seq.empty)
543+
}
544+
}
545+
546+
test("LEFT function with dictionary") {
547+
val data = (0 until 1000)
548+
.map(_ % 5)
549+
.map(i => s"value$i")
550+
withParquetTable(data.zipWithIndex, "dict_tbl") {
551+
checkSparkAnswerAndOperator("SELECT _1, LEFT(_1, 3) FROM dict_tbl")
552+
}
553+
}
554+
509555
test("hour, minute, second") {
510556
Seq(true, false).foreach { dictionaryEnabled =>
511557
withTempDir { dir =>

0 commit comments

Comments
 (0)