1919
2020package org .apache .comet .serde
2121
22- import org .apache .spark .sql .catalyst .expressions .Attribute
22+ import org .apache .spark .sql .catalyst .expressions .{ Attribute , ExpressionImplUtils }
2323import org .apache .spark .sql .catalyst .expressions .objects .StaticInvoke
2424import org .apache .spark .sql .catalyst .util .CharVarcharCodegenUtils
25+ import org .apache .spark .sql .types .BooleanType
2526
2627import org .apache .comet .CometSparkSessionExtensions .withInfo
28+ import org .apache .comet .serde .QueryPlanSerde .{exprToProtoInternal , optExprWithInfo , scalarFunctionExprToProtoWithReturnType }
2729
2830object CometStaticInvoke extends CometExpressionSerde [StaticInvoke ] {
2931
@@ -34,7 +36,8 @@ object CometStaticInvoke extends CometExpressionSerde[StaticInvoke] {
3436 : Map [(String , Class [_]), CometExpressionSerde [StaticInvoke ]] =
3537 Map (
3638 (" readSidePadding" , classOf [CharVarcharCodegenUtils ]) -> CometScalarFunction (
37- " read_side_padding" ))
39+ " read_side_padding" ),
40+ (" isLuhnNumber" , classOf [ExpressionImplUtils ]) -> CometLuhnCheck )
3841
3942 override def convert (
4043 expr : StaticInvoke ,
@@ -52,3 +55,23 @@ object CometStaticInvoke extends CometExpressionSerde[StaticInvoke] {
5255 }
5356 }
5457}
58+
59+ /**
60+ * Handler for ExpressionImplUtils.isLuhnNumber StaticInvoke (Spark 3.5+).
61+ * Maps to datafusion-spark's built-in luhn_check function.
62+ */
63+ private object CometLuhnCheck extends CometExpressionSerde [StaticInvoke ] {
64+
65+ override def convert (
66+ expr : StaticInvoke ,
67+ inputs : Seq [Attribute ],
68+ binding : Boolean ): Option [ExprOuterClass .Expr ] = {
69+ val childExpr = exprToProtoInternal(expr.arguments.head, inputs, binding)
70+ val optExpr = scalarFunctionExprToProtoWithReturnType(
71+ " luhn_check" ,
72+ BooleanType ,
73+ false ,
74+ childExpr)
75+ optExprWithInfo(optExpr, expr, expr.arguments.head)
76+ }
77+ }
0 commit comments