forked from apache/datafusion-comet
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathstrings.scala
More file actions
437 lines (387 loc) · 15.3 KB
/
strings.scala
File metadata and controls
437 lines (387 loc) · 15.3 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.comet.serde
import java.util.Locale
import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, ConcatWs, Expression, GetJsonObject, If, 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
import org.apache.comet.CometConf
import org.apache.comet.CometSparkSessionExtensions.withInfo
import org.apache.comet.expressions.{CometCast, CometEvalMode, RegExp}
import org.apache.comet.serde.ExprOuterClass.Expr
import org.apache.comet.serde.QueryPlanSerde.{createBinaryExpr, exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto, scalarFunctionExprToProtoWithReturnType}
object CometStringRepeat extends CometExpressionSerde[StringRepeat] {
override def convert(
expr: StringRepeat,
inputs: Seq[Attribute],
binding: Boolean): Option[ExprOuterClass.Expr] = {
val children = expr.children
val leftCast = Cast(children(0), StringType)
val rightCast = Cast(children(1), LongType)
val leftExpr = exprToProtoInternal(leftCast, inputs, binding)
val rightExpr = exprToProtoInternal(rightCast, inputs, binding)
val optExpr = scalarFunctionExprToProto("repeat", leftExpr, rightExpr)
optExprWithInfo(optExpr, expr, leftCast, rightCast)
}
}
class CometCaseConversionBase[T <: Expression](function: String)
extends CometScalarFunction[T](function) {
override def convert(expr: T, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
if (!CometConf.COMET_CASE_CONVERSION_ENABLED.get()) {
withInfo(
expr,
"Comet is not compatible with Spark for case conversion in " +
s"locale-specific cases. Set ${CometConf.COMET_CASE_CONVERSION_ENABLED.key}=true " +
"to enable it anyway.")
return None
}
super.convert(expr, inputs, binding)
}
}
object CometUpper extends CometCaseConversionBase[Upper]("upper")
object CometLower extends CometCaseConversionBase[Lower]("lower")
object CometLength extends CometScalarFunction[Length]("length") {
override val conditions: Seq[SupportCondition[Length]] = Seq(
SupportCondition.unsupported[Length](
id = "binary-child",
description = "Child is BinaryType",
fires = _.child.dataType.isInstanceOf[BinaryType],
message = "Length on BinaryType is not supported"))
}
object CometInitCap extends CometScalarFunction[InitCap]("initcap") {
override def getSupportLevel(expr: InitCap): SupportLevel = {
// Behavior differs from Spark. One example is that for the input "robert rose-smith", Spark
// will produce "Robert Rose-smith", but Comet will produce "Robert Rose-Smith".
// https://github.com/apache/datafusion-comet/issues/1052
Incompatible(None)
}
override def convert(expr: InitCap, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
super.convert(expr, inputs, binding)
}
}
object CometSubstring extends CometExpressionSerde[Substring] {
override def convert(
expr: Substring,
inputs: Seq[Attribute],
binding: Boolean): Option[Expr] = {
(expr.pos, expr.len) match {
case (Literal(pos, _), Literal(len, _)) =>
exprToProtoInternal(expr.str, inputs, binding) match {
case Some(strExpr) =>
val builder = ExprOuterClass.Substring.newBuilder()
builder.setChild(strExpr)
builder.setStart(pos.asInstanceOf[Int])
builder.setLen(len.asInstanceOf[Int])
Some(ExprOuterClass.Expr.newBuilder().setSubstring(builder).build())
case None =>
withInfo(expr, expr.str)
None
}
case _ =>
withInfo(expr, "Substring pos and len must be literals")
None
}
}
}
object CometLeft extends CometExpressionSerde[Left] {
override def convert(expr: Left, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
expr.len match {
case Literal(lenValue, _) =>
exprToProtoInternal(expr.str, inputs, binding) match {
case Some(strExpr) =>
val builder = ExprOuterClass.Substring.newBuilder()
builder.setChild(strExpr)
builder.setStart(1)
builder.setLen(lenValue.asInstanceOf[Int])
Some(ExprOuterClass.Expr.newBuilder().setSubstring(builder).build())
case None =>
withInfo(expr, expr.str)
None
}
case _ =>
withInfo(expr, "LEFT len must be a literal")
None
}
}
override def getSupportLevel(expr: Left): SupportLevel = {
expr.str.dataType match {
case _: BinaryType | _: StringType => Compatible()
case _ => Unsupported(Some(s"LEFT does not support ${expr.str.dataType}"))
}
}
}
object CometRight extends CometExpressionSerde[Right] {
override def convert(expr: Right, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
expr.len match {
case Literal(lenValue, _) =>
val lenInt = lenValue.asInstanceOf[Int]
if (lenInt <= 0) {
// Match Spark's behavior: If(IsNull(str), NULL, "")
// This ensures NULL propagation: RIGHT(NULL, 0) -> NULL, RIGHT("hello", 0) -> ""
val isNullExpr = IsNull(expr.str)
val nullLiteral = Literal.create(null, StringType)
val emptyStringLiteral = Literal(UTF8String.EMPTY_UTF8, StringType)
val ifExpr = If(isNullExpr, nullLiteral, emptyStringLiteral)
// Serialize the If expression using existing infrastructure
exprToProtoInternal(ifExpr, inputs, binding)
} else {
exprToProtoInternal(expr.str, inputs, binding) match {
case Some(strExpr) =>
val builder = ExprOuterClass.Substring.newBuilder()
builder.setChild(strExpr)
builder.setStart(-lenInt)
builder.setLen(lenInt)
Some(ExprOuterClass.Expr.newBuilder().setSubstring(builder).build())
case None =>
withInfo(expr, expr.str)
None
}
}
case _ =>
withInfo(expr, "RIGHT len must be a literal")
None
}
}
override def getSupportLevel(expr: Right): SupportLevel = {
expr.str.dataType match {
case _: StringType => Compatible()
case _ => Unsupported(Some(s"RIGHT does not support ${expr.str.dataType}"))
}
}
}
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 CometConcatWs extends CometExpressionSerde[ConcatWs] {
override def convert(expr: ConcatWs, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
expr.children.headOption match {
// Match Spark behavior: when the separator is NULL, the result of concat_ws is NULL.
case Some(Literal(null, _)) =>
val nullLiteral = Literal.create(null, expr.dataType)
exprToProtoInternal(nullLiteral, inputs, binding)
case _ if expr.children.forall(_.foldable) =>
// Fall back to Spark for all-literal args so ConstantFolding can handle it
withInfo(expr, "all arguments are foldable")
None
case _ =>
// For all other cases, use the generic scalar function implementation.
CometScalarFunction[ConcatWs]("concat_ws").convert(expr, inputs, binding)
}
}
}
object CometLike extends CometExpressionSerde[Like] {
override def convert(expr: Like, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
if (expr.escapeChar == '\\') {
createBinaryExpr(
expr,
expr.left,
expr.right,
inputs,
binding,
(builder, binaryExpr) => builder.setLike(binaryExpr))
} else {
withInfo(expr, s"custom escape character ${expr.escapeChar} not supported in LIKE")
None
}
}
}
object CometRLike extends CometExpressionSerde[RLike] {
override def convert(expr: RLike, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
expr.right match {
case Literal(pattern, DataTypes.StringType) =>
if (!RegExp.isSupportedPattern(pattern.toString) &&
!CometConf.isExprAllowIncompat("regexp")) {
withInfo(
expr,
s"Regexp pattern $pattern is not compatible with Spark. " +
s"Set ${CometConf.getExprAllowIncompatConfigKey("regexp")}=true " +
"to allow it anyway.")
None
} else {
createBinaryExpr(
expr,
expr.left,
expr.right,
inputs,
binding,
(builder, binaryExpr) => builder.setRlike(binaryExpr))
}
case _ =>
withInfo(expr, "Only scalar regexp patterns are supported")
None
}
}
}
object CometStringRPad extends CometExpressionSerde[StringRPad] {
override def getSupportLevel(expr: StringRPad): SupportLevel = {
if (expr.str.isInstanceOf[Literal]) {
return Unsupported(Some("Scalar values are not supported for the str argument"))
}
if (!expr.pad.isInstanceOf[Literal]) {
return Unsupported(Some("Only scalar values are supported for the pad argument"))
}
Compatible()
}
override def convert(
expr: StringRPad,
inputs: Seq[Attribute],
binding: Boolean): Option[Expr] = {
scalarFunctionExprToProto(
"rpad",
exprToProtoInternal(expr.str, inputs, binding),
exprToProtoInternal(expr.len, inputs, binding),
exprToProtoInternal(expr.pad, inputs, binding))
}
}
object CometStringLPad extends CometExpressionSerde[StringLPad] {
override def getSupportLevel(expr: StringLPad): SupportLevel = {
if (expr.str.isInstanceOf[Literal]) {
return Unsupported(Some("Scalar values are not supported for the str argument"))
}
if (!expr.pad.isInstanceOf[Literal]) {
return Unsupported(Some("Only scalar values are supported for the pad argument"))
}
Compatible()
}
override def convert(
expr: StringLPad,
inputs: Seq[Attribute],
binding: Boolean): Option[Expr] = {
scalarFunctionExprToProto(
"lpad",
exprToProtoInternal(expr.str, inputs, binding),
exprToProtoInternal(expr.len, inputs, binding),
exprToProtoInternal(expr.pad, inputs, binding))
}
}
object CometRegExpReplace extends CometExpressionSerde[RegExpReplace] {
override def getSupportLevel(expr: RegExpReplace): SupportLevel = {
if (!RegExp.isSupportedPattern(expr.regexp.toString) &&
!CometConf.isExprAllowIncompat("regexp")) {
withInfo(
expr,
s"Regexp pattern ${expr.regexp} is not compatible with Spark. " +
s"Set ${CometConf.getExprAllowIncompatConfigKey("regexp")}=true " +
"to allow it anyway.")
return Incompatible()
}
expr.pos match {
case Literal(value, DataTypes.IntegerType) if value == 1 => Compatible()
case _ =>
Unsupported(Some("Comet only supports regexp_replace with an offset of 1 (no offset)."))
}
}
override def convert(
expr: RegExpReplace,
inputs: Seq[Attribute],
binding: Boolean): Option[Expr] = {
val subjectExpr = exprToProtoInternal(expr.subject, inputs, binding)
val patternExpr = exprToProtoInternal(expr.regexp, inputs, binding)
val replacementExpr = exprToProtoInternal(expr.rep, inputs, binding)
// DataFusion's regexp_replace stops at the first match. We need to add the 'g' flag
// to apply the regex globally to match Spark behavior.
val flagsExpr = exprToProtoInternal(Literal("g"), inputs, binding)
val optExpr = scalarFunctionExprToProto(
"regexp_replace",
subjectExpr,
patternExpr,
replacementExpr,
flagsExpr)
optExprWithInfo(optExpr, expr, expr.subject, expr.regexp, expr.rep, expr.pos)
}
}
/**
* Serde for StringSplit expression. This is a custom Comet function (not a built-in DataFusion
* function), so we need to include the return type in the protobuf to avoid DataFusion registry
* lookup failures.
*/
object CometStringSplit extends CometExpressionSerde[StringSplit] {
override def getSupportLevel(expr: StringSplit): SupportLevel =
Incompatible(Some("Regex engine differences between Java and Rust"))
override def convert(
expr: StringSplit,
inputs: Seq[Attribute],
binding: Boolean): Option[Expr] = {
val strExpr = exprToProtoInternal(expr.str, inputs, binding)
val regexExpr = exprToProtoInternal(expr.regex, inputs, binding)
val limitExpr = exprToProtoInternal(expr.limit, inputs, binding)
val optExpr = scalarFunctionExprToProtoWithReturnType(
"split",
expr.dataType,
false,
strExpr,
regexExpr,
limitExpr)
optExprWithInfo(optExpr, expr, expr.str, expr.regex, expr.limit)
}
}
object CometGetJsonObject extends CometExpressionSerde[GetJsonObject] {
override def getSupportLevel(expr: GetJsonObject): SupportLevel =
Incompatible(
Some(
"Spark allows single-quoted JSON and unescaped control characters " +
"which Comet does not support"))
override def convert(
expr: GetJsonObject,
inputs: Seq[Attribute],
binding: Boolean): Option[Expr] = {
val jsonExpr = exprToProtoInternal(expr.json, inputs, binding)
val pathExpr = exprToProtoInternal(expr.path, inputs, binding)
val optExpr = scalarFunctionExprToProtoWithReturnType(
"get_json_object",
expr.dataType,
false,
jsonExpr,
pathExpr)
optExprWithInfo(optExpr, expr, expr.json, expr.path)
}
}
trait CommonStringExprs {
def stringDecode(
expr: Expression,
charset: Expression,
bin: Expression,
inputs: Seq[Attribute],
binding: Boolean): Option[Expr] = {
charset match {
case Literal(str, DataTypes.StringType)
if str.toString.toLowerCase(Locale.ROOT) == "utf-8" =>
// decode(col, 'utf-8') can be treated as a cast with "try" eval mode that puts nulls
// for invalid strings.
// Left child is the binary expression.
val binExpr = exprToProtoInternal(bin, inputs, binding)
if (binExpr.isDefined) {
CometCast.castToProto(expr, None, DataTypes.StringType, binExpr.get, CometEvalMode.TRY)
} else {
withInfo(expr, bin)
None
}
case _ =>
withInfo(expr, "Comet only supports decoding with 'utf-8'.")
None
}
}
}