This is an automated email from the ASF dual-hosted git repository.

philo pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
     new 0adfec8c1d [GLUTEN-9535][FLINK] Refactor the conversion for RexCall 
(#9536)
0adfec8c1d is described below

commit 0adfec8c1d5312c0d2b18bef7c6e24fdce3c3410
Author: lgbo <[email protected]>
AuthorDate: Wed May 21 13:11:30 2025 +0800

    [GLUTEN-9535][FLINK] Refactor the conversion for RexCall (#9536)
---
 .../plan/nodes/exec/stream/StreamExecCalc.java     |   6 +-
 .../exec/stream/StreamExecWatermarkAssigner.java   |   5 +-
 .../apache/gluten/rexnode/FunctionMappings.java    |  51 ----------
 ...ionConverter.java => RexConversionContext.java} |  21 ++---
 .../apache/gluten/rexnode/RexNodeConverter.java    |  19 ++--
 .../main/java/org/apache/gluten/rexnode/Utils.java |  43 +++++++++
 .../rexnode/functions/BaseRexCallConverters.java   |  72 ++++++++++++++
 ...tionConverter.java => ModRexCallConverter.java} |  25 +++--
 ...unctionConverter.java => RexCallConverter.java} |  17 +++-
 .../rexnode/functions/RexCallConverterFactory.java |  49 ++++++++++
 .../functions/SubtractFunctionConverter.java       |  56 -----------
 .../runtime/stream/custom/ScalarFunctionsTest.java | 104 +++++++++++++++++++++
 12 files changed, 321 insertions(+), 147 deletions(-)

diff --git 
a/gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
 
b/gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
index 143dd888cc..5b346738d4 100644
--- 
a/gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
+++ 
b/gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream;
 import org.apache.gluten.rexnode.Utils;
 import org.apache.gluten.table.runtime.operators.GlutenSingleInputOperator;
 import org.apache.gluten.util.LogicalTypeConverter;
+import org.apache.gluten.rexnode.RexConversionContext;
 import org.apache.gluten.rexnode.RexNodeConverter;
 
 import io.github.zhztheplayer.velox4j.expression.TypedExpr;
@@ -116,14 +117,15 @@ public class StreamExecCalc extends CommonExecCalc 
implements StreamExecNode<Row
                 (io.github.zhztheplayer.velox4j.type.RowType)
                         
LogicalTypeConverter.toVLType(inputEdge.getOutputType());
         List<String> inNames = 
Utils.getNamesFromRowType(inputEdge.getOutputType());
+        RexConversionContext conversionContext = new 
RexConversionContext(inNames);
         PlanNode filter = null;
         if (condition != null) {
             filter = new FilterNode(
                     PlanNodeIdGenerator.newId(),
                     List.of(),
-                    RexNodeConverter.toTypedExpr(condition, inNames));
+                    RexNodeConverter.toTypedExpr(condition, 
conversionContext));
         }
-        List<TypedExpr> projectExprs = 
RexNodeConverter.toTypedExpr(projection, inNames);
+        List<TypedExpr> projectExprs = 
RexNodeConverter.toTypedExpr(projection, conversionContext);
         PlanNode project = new ProjectNode(
                 PlanNodeIdGenerator.newId(),
                 filter == null ? List.of() : List.of(filter),
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
 
b/gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
index b02f74c2d4..863fee7313 100644
--- 
a/gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
+++ 
b/gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
@@ -45,6 +45,7 @@ import io.github.zhztheplayer.velox4j.plan.PlanNode;
 import io.github.zhztheplayer.velox4j.plan.TableScanNode;
 import io.github.zhztheplayer.velox4j.plan.WatermarkAssignerNode;
 import org.apache.calcite.rex.RexNode;
+import org.apache.gluten.rexnode.RexConversionContext;
 import org.apache.gluten.rexnode.RexNodeConverter;
 import org.apache.gluten.rexnode.Utils;
 import org.apache.gluten.table.runtime.operators.GlutenSingleInputOperator;
@@ -128,8 +129,8 @@ public class StreamExecWatermarkAssigner extends 
ExecNodeBase<RowData>
                 (io.github.zhztheplayer.velox4j.type.RowType)
                         
LogicalTypeConverter.toVLType(inputEdge.getOutputType());
         List<String> inNames = 
Utils.getNamesFromRowType(inputEdge.getOutputType());
-
-        TypedExpr watermarkExprs = RexNodeConverter.toTypedExpr(watermarkExpr, 
inNames);
+        RexConversionContext conversionContext = new 
RexConversionContext(inNames);
+        TypedExpr watermarkExprs = RexNodeConverter.toTypedExpr(watermarkExpr, 
conversionContext);
         io.github.zhztheplayer.velox4j.type.RowType outputType =
                 (io.github.zhztheplayer.velox4j.type.RowType)
                         LogicalTypeConverter.toVLType(getOutputType());
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/FunctionMappings.java
 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/FunctionMappings.java
deleted file mode 100644
index 79915bb5e6..0000000000
--- 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/FunctionMappings.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.gluten.rexnode;
-
-import org.apache.gluten.rexnode.functions.DefaultFunctionConverter;
-import org.apache.gluten.rexnode.functions.FunctionConverter;
-import org.apache.gluten.rexnode.functions.SubtractFunctionConverter;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/** Mapping of flink function and velox function. */
-public class FunctionMappings {
-    // A map stores the relationship between flink function name and velox 
function.
-    private static Map<String, FunctionConverter> functionMappings = new 
HashMap() {
-        {
-            // TODO: support more functions.
-            put(">", new DefaultFunctionConverter("greaterthan"));
-            put("<", new DefaultFunctionConverter("lessthan"));
-            put("=", new DefaultFunctionConverter("equalto"));
-            put("CAST", new DefaultFunctionConverter("cast"));
-            put("CASE", new DefaultFunctionConverter("if"));
-            put("*", new DefaultFunctionConverter("multiply"));
-            put("-", new SubtractFunctionConverter("subtract"));
-            put("MOD", new DefaultFunctionConverter("remainder"));
-            put("AND", new DefaultFunctionConverter("and"));
-        }
-    };
-
-    public static FunctionConverter getFunctionConverter(String funcName) {
-        if (functionMappings.containsKey(funcName)) {
-            return functionMappings.get(funcName);
-        } else {
-            throw new RuntimeException("Function not supported: " + funcName);
-        }
-    }
-}
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/DefaultFunctionConverter.java
 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/RexConversionContext.java
similarity index 57%
copy from 
gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/DefaultFunctionConverter.java
copy to 
gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/RexConversionContext.java
index 5f1d35bc38..3053cf3dc4 100644
--- 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/DefaultFunctionConverter.java
+++ 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/RexConversionContext.java
@@ -14,24 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.gluten.rexnode.functions;
 
-import io.github.zhztheplayer.velox4j.expression.CallTypedExpr;
-import io.github.zhztheplayer.velox4j.expression.TypedExpr;
-import io.github.zhztheplayer.velox4j.type.Type;
+package org.apache.gluten.rexnode;
 
 import java.util.List;
 
-/** Default convertor for velox function. */
-public class DefaultFunctionConverter implements FunctionConverter {
-    private final String function;
+// In case more information is needed in the future, we can add more fields to 
this class.
+public class RexConversionContext {
+    private final List<String> inputAttributeNames;
 
-    public DefaultFunctionConverter(String function) {
-        this.function = function;
+    public RexConversionContext(List<String> inputAttributeNames) {
+        this.inputAttributeNames = inputAttributeNames;
     }
-
-    @Override
-    public CallTypedExpr toVeloxFunction(Type nodeType, List<TypedExpr> 
params) {
-        return new CallTypedExpr(nodeType, params, function);
+    public List<String> getInputAttributeNames() {
+        return inputAttributeNames;
     }
 }
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/RexNodeConverter.java
 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/RexNodeConverter.java
index 43ce8839f2..f401023395 100644
--- 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/RexNodeConverter.java
+++ 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/RexNodeConverter.java
@@ -38,6 +38,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.gluten.rexnode.RexConversionContext;
 import org.apache.gluten.util.LogicalTypeConverter;
 
 import java.math.BigDecimal;
@@ -47,7 +48,7 @@ import java.util.stream.Collectors;
 /** Convertor to convert RexNode to velox TypedExpr */
 public class RexNodeConverter {
 
-    public static TypedExpr toTypedExpr(RexNode rexNode, List<String> inNames) 
{
+    public static TypedExpr toTypedExpr(RexNode rexNode, RexConversionContext 
context) {
         if (rexNode instanceof RexLiteral) {
             RexLiteral literal = (RexLiteral) rexNode;
             return new ConstantTypedExpr(
@@ -56,28 +57,28 @@ public class RexNodeConverter {
                     null);
         } else if (rexNode instanceof RexCall) {
             RexCall rexCall = (RexCall) rexNode;
-            List<TypedExpr> params = toTypedExpr(rexCall.getOperands(), 
inNames);
-            Type nodeType = toType(rexCall.getType());
-            return 
FunctionMappings.getFunctionConverter(rexCall.getOperator().getName())
-                    .toVeloxFunction(nodeType, params);
+            String operatorName = rexCall.getOperator().getName();
+            RexCallConverter converter = 
RexCallConverterFactory.getConverter(operatorName);
+            return converter.toTypedExpr(rexCall, context);
         } else if (rexNode instanceof RexInputRef) {
             RexInputRef inputRef = (RexInputRef) rexNode;
+            List<String> inputAttributes = context.getInputAttributeNames();
             return FieldAccessTypedExpr.create(
                     toType(inputRef.getType()),
-                    inNames.get(inputRef.getIndex()));
+                    inputAttributes.get(inputRef.getIndex()));
         } else if (rexNode instanceof RexFieldAccess) {
             RexFieldAccess fieldAccess = (RexFieldAccess) rexNode;
             return FieldAccessTypedExpr.create(
-                    toTypedExpr(fieldAccess.getReferenceExpr(), inNames),
+                    toTypedExpr(fieldAccess.getReferenceExpr(), context),
                     fieldAccess.getField().getName());
         } else {
             throw new RuntimeException("Unrecognized RexNode: " + 
rexNode.getClass().getName());
         }
     }
 
-    public static List<TypedExpr> toTypedExpr(List<RexNode> rexNodes, 
List<String> inNames) {
+    public static List<TypedExpr> toTypedExpr(List<RexNode> rexNodes, 
RexConversionContext context) {
         return rexNodes.stream()
-                .map(rexNode -> toTypedExpr(rexNode, inNames))
+                .map(rexNode -> toTypedExpr(rexNode, context))
                 .collect(Collectors.toList());
     }
 
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/Utils.java 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/Utils.java
index 6072b7b85e..ad71ec2b0f 100644
--- a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/Utils.java
+++ b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/Utils.java
@@ -16,15 +16,25 @@
  */
 package org.apache.gluten.rexnode;
 
+import io.github.zhztheplayer.velox4j.expression.CastTypedExpr;
+import io.github.zhztheplayer.velox4j.expression.TypedExpr;
 import io.github.zhztheplayer.velox4j.serializable.ISerializableRegistry;
+import io.github.zhztheplayer.velox4j.type.*;
 import io.github.zhztheplayer.velox4j.variant.VariantRegistry;
+
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 
 import java.util.List;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** Utility to store some useful functions. */
 public class Utils {
+    private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
 
     private static boolean registryInitialized = false;
 
@@ -46,4 +56,37 @@ public class Utils {
             ISerializableRegistry.registerAll();
         }
     }
+
+    private static final List<Class<?>> NUMERIC_TYPE_PRIORITY_LIST = List.of(
+        TinyIntType.class,
+        SmallIntType.class,
+        IntegerType.class,
+        BigIntType.class,
+        RealType.class,
+        DoubleType.class
+    );
+
+    private static int getNumericTypePriority(Type type) {
+        int index = NUMERIC_TYPE_PRIORITY_LIST.indexOf(type.getClass());
+        if (index == -1) {
+            // If the type is not found in the list, throw an exception
+            throw new RuntimeException("Unsupported type: " + 
type.getClass().getName());
+        }
+        return index;
+    }
+
+    public static List<TypedExpr> 
promoteTypeForArithmeticExpressions(List<TypedExpr> expressions) {
+        Type targetType = expressions.stream()
+            .map(expr -> {
+                Type returnType = expr.getReturnType();
+                int priority = getNumericTypePriority(returnType);
+            return new Tuple2<>(priority, returnType);
+        })
+        .max((t1, t2) -> Integer.compare(t1.f0, t2.f0))
+        .orElseThrow(() -> new RuntimeException("No expressions found")).f1;
+
+        return expressions.stream()
+            .map(expr -> expr.getReturnType().equals(targetType) ? expr : 
CastTypedExpr.create(targetType, expr, false))
+            .collect(Collectors.toList());
+    }
 }
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/BaseRexCallConverters.java
 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/BaseRexCallConverters.java
new file mode 100644
index 0000000000..f106d538d6
--- /dev/null
+++ 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/BaseRexCallConverters.java
@@ -0,0 +1,72 @@
+/*
+ * 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.gluten.rexnode;
+
+import io.github.zhztheplayer.velox4j.expression.CallTypedExpr;
+import io.github.zhztheplayer.velox4j.expression.TypedExpr;
+import io.github.zhztheplayer.velox4j.type.Type;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.gluten.rexnode.RexConversionContext;
+import org.apache.gluten.rexnode.RexNodeConverter;
+
+import java.util.List;
+
+abstract class BaseRexCallConverter implements RexCallConverter {
+    protected final String functionName;
+
+    public BaseRexCallConverter(String functionName) {
+        this.functionName = functionName;
+    }
+
+    protected List<TypedExpr> getParams(RexCall callNode, RexConversionContext 
context) {
+        return RexNodeConverter.toTypedExpr(callNode.getOperands(), context);
+    }
+
+    protected Type getResultType(RexCall callNode) {
+        return RexNodeConverter.toType(callNode.getType());
+    }
+}
+
+class DefaultRexCallConverter extends BaseRexCallConverter {
+    public DefaultRexCallConverter(String functionName) {
+        super(functionName);
+    }
+
+    @Override
+    public TypedExpr toTypedExpr(RexCall callNode, RexConversionContext 
context) {
+        List<TypedExpr> params = getParams(callNode, context);
+        Type resultType = getResultType(callNode);
+        return new CallTypedExpr(resultType, params, functionName);
+    }
+}
+
+class AlignInputsTypeRexCallConverter extends BaseRexCallConverter {
+    public AlignInputsTypeRexCallConverter(String functionName) {
+        super(functionName);
+    }
+
+    @Override
+    public TypedExpr toTypedExpr(RexCall callNode, RexConversionContext 
context) {
+        List<TypedExpr> params = getParams(callNode, context);
+        List<TypedExpr> alignedParams = 
Utils.promoteTypeForArithmeticExpressions(params);
+        Type resultType = getResultType(callNode);
+        return new CallTypedExpr(resultType, alignedParams, functionName);
+    }
+}
+
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/DefaultFunctionConverter.java
 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/ModRexCallConverter.java
similarity index 56%
rename from 
gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/DefaultFunctionConverter.java
rename to 
gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/ModRexCallConverter.java
index 5f1d35bc38..92709d1d7a 100644
--- 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/DefaultFunctionConverter.java
+++ 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/ModRexCallConverter.java
@@ -14,24 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.gluten.rexnode.functions;
+
+package org.apache.gluten.rexnode;
 
 import io.github.zhztheplayer.velox4j.expression.CallTypedExpr;
 import io.github.zhztheplayer.velox4j.expression.TypedExpr;
 import io.github.zhztheplayer.velox4j.type.Type;
 
-import java.util.List;
+import org.apache.calcite.rex.RexCall;
+import org.apache.gluten.rexnode.RexConversionContext;
+import org.apache.gluten.rexnode.RexNodeConverter;
 
-/** Default convertor for velox function. */
-public class DefaultFunctionConverter implements FunctionConverter {
-    private final String function;
+import java.util.List;
 
-    public DefaultFunctionConverter(String function) {
-        this.function = function;
+ public class ModRexCallConverter extends BaseRexCallConverter {
+    private static final String FUNCTION_NAME = "remainder";
+    public ModRexCallConverter() {
+        super(FUNCTION_NAME);
     }
 
     @Override
-    public CallTypedExpr toVeloxFunction(Type nodeType, List<TypedExpr> 
params) {
-        return new CallTypedExpr(nodeType, params, function);
+    public TypedExpr toTypedExpr(RexCall callNode, RexConversionContext 
context) {
+        List<TypedExpr> params = getParams(callNode, context);
+        List<TypedExpr> alignedParams = 
Utils.promoteTypeForArithmeticExpressions(params);
+        // Use the divisor's type as the result type
+        Type resultType = params.get(1).getReturnType();
+        return new CallTypedExpr(resultType, params, functionName);
     }
 }
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/FunctionConverter.java
 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/RexCallConverter.java
similarity index 70%
rename from 
gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/FunctionConverter.java
rename to 
gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/RexCallConverter.java
index 7f29adfeb0..2a01ee85c1 100644
--- 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/FunctionConverter.java
+++ 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/RexCallConverter.java
@@ -14,16 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.gluten.rexnode.functions;
+
+package org.apache.gluten.rexnode;
 
 import io.github.zhztheplayer.velox4j.expression.CallTypedExpr;
 import io.github.zhztheplayer.velox4j.expression.TypedExpr;
 import io.github.zhztheplayer.velox4j.type.Type;
 
-import java.util.List;
+import org.apache.calcite.rex.RexCall;
+import org.apache.gluten.rexnode.RexConversionContext;
+import org.apache.gluten.rexnode.RexNodeConverter;
 
-/** Interface for converter to velox function. */
-public interface FunctionConverter {
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
 
-    CallTypedExpr toVeloxFunction(Type nodeType, List<TypedExpr> params);
+public interface RexCallConverter {
+   // Let the Converter to decide how to build the arguments.
+   TypedExpr toTypedExpr(RexCall callNode, RexConversionContext context);
 }
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/RexCallConverterFactory.java
 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/RexCallConverterFactory.java
new file mode 100644
index 0000000000..ef8a55c6ec
--- /dev/null
+++ 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/RexCallConverterFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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.gluten.rexnode;
+
+import java.util.Map;
+
+public class RexCallConverterFactory {
+
+    // Better to new Converter for each call. Reusing an object can easily 
introduce potential
+    // issues.
+    private static Map<String, RexCallConverterBuilder> converters = Map.of(
+        ">", () -> new AlignInputsTypeRexCallConverter("greaterthan"),
+        "<", () -> new AlignInputsTypeRexCallConverter("lessthan"),
+        "=", () -> new AlignInputsTypeRexCallConverter("equalto"),
+        "*", () -> new AlignInputsTypeRexCallConverter("multiply"),
+        "-", () -> new AlignInputsTypeRexCallConverter("subtract"),
+        "+", () -> new AlignInputsTypeRexCallConverter("add"),
+        "MOD", () -> new ModRexCallConverter(),
+        "CAST", () -> new DefaultRexCallConverter("cast"),
+        "CASE", () -> new DefaultRexCallConverter("if"),
+        "AND", () -> new DefaultRexCallConverter("and")
+    );
+
+    public static RexCallConverter getConverter(String operatorName) {
+        RexCallConverterBuilder builder = converters.get(operatorName);
+        if (builder == null) {
+            throw new RuntimeException("Function not supported: " + 
operatorName);
+        }
+        return builder.build();
+    }
+
+    private interface RexCallConverterBuilder {
+        RexCallConverter build();
+    }
+}
diff --git 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/SubtractFunctionConverter.java
 
b/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/SubtractFunctionConverter.java
deleted file mode 100644
index 483c5cf1ac..0000000000
--- 
a/gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/SubtractFunctionConverter.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.gluten.rexnode.functions;
-
-import io.github.zhztheplayer.velox4j.expression.CallTypedExpr;
-import io.github.zhztheplayer.velox4j.expression.TypedExpr;
-import io.github.zhztheplayer.velox4j.type.BigIntType;
-import io.github.zhztheplayer.velox4j.type.TimestampType;
-import io.github.zhztheplayer.velox4j.type.Type;
-
-import org.apache.flink.util.Preconditions;
-import java.util.List;
-
-/** Subtract function converter. */
-public class SubtractFunctionConverter implements FunctionConverter {
-    private final String function;
-
-    public SubtractFunctionConverter(String function) {
-        this.function = function;
-    }
-
-    @Override
-    public CallTypedExpr toVeloxFunction(Type nodeType, List<TypedExpr> 
params) {
-        Preconditions.checkNotNull(params.size() == 2, "Subtract must contain 
exactly two parameters");
-
-        // TODO: need refine for more type cast
-        if (params.get(0).getReturnType().getClass() == TimestampType.class &&
-                params.get(1).getReturnType().getClass() == BigIntType.class) {
-            // hardcode here for next mark watermark whose param1 is Timestamp 
and 2 is BigInt.
-            Type newType = new BigIntType();
-            TypedExpr param0 = new CallTypedExpr(
-                    newType,
-                    List.of(params.get(0)),
-                    "cast");
-            return new CallTypedExpr(
-                    newType,
-                    List.of(param0, params.get(1)),
-                    function);
-        }
-        return new CallTypedExpr(nodeType, params, function);
-    }
-}
diff --git 
a/gluten-flink/ut/src/test/java/org/apache/gluten/table/runtime/stream/custom/ScalarFunctionsTest.java
 
b/gluten-flink/ut/src/test/java/org/apache/gluten/table/runtime/stream/custom/ScalarFunctionsTest.java
new file mode 100644
index 0000000000..af2ba9108c
--- /dev/null
+++ 
b/gluten-flink/ut/src/test/java/org/apache/gluten/table/runtime/stream/custom/ScalarFunctionsTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.gluten.table.runtime.stream.custom;
+
+import org.apache.gluten.table.runtime.stream.common.GlutenStreamingTestBase;
+import org.apache.flink.types.Row;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+
+class ScalarFunctionsTest extends GlutenStreamingTestBase {
+
+    @Override
+    @BeforeEach
+    public void before() throws Exception {
+        super.before();
+        List<Row> rows =
+                Arrays.asList(Row.of(1, 1L, "1", 100, 10L),
+                    Row.of(2, 2L, "2", 3, 11L),
+                    Row.of(3, 3L, "3", 5, 12L));
+        createSimpleBoundedValuesTable("t1", "a int, b bigint, c string, d 
int, e bigint", rows);
+
+        List<Row> rows2 =
+                Arrays.asList(Row.of(1, 2L, 3.0, "12"),
+                    Row.of(2, 3L, 4.0, "13"),
+                    Row.of(3, 4L, 5.0, "14"));
+        createSimpleBoundedValuesTable("t2", "a int, b bigint, c double, d 
string", rows2);
+    }
+
+    @Test
+    void testAdd() {
+        String query1 = "select a + b as x from t1 where a > 0";
+        runAndCheck(query1, Arrays.asList("+I[2]", "+I[4]", "+I[6]"));
+
+        String query2 = "select a + 1 as x from t1 where a > 0";
+        runAndCheck(query2, Arrays.asList("+I[2]", "+I[3]", "+I[4]"));
+
+    }
+
+    @Test
+    void testSubtract() {
+        String query1 = "select a - b as x from t1 where a > 0";
+        runAndCheck(query1, Arrays.asList("+I[0]", "+I[0]", "+I[0]"));
+
+        String query2 = "select a - 1 as x from t1 where a > 0";
+        runAndCheck(query2, Arrays.asList("+I[0]", "+I[1]", "+I[2]"));
+    }
+
+    @Test
+    void testMod() {
+        String query1 = "select d % a as x from t1 where a > 0";
+        runAndCheck(query1, Arrays.asList("+I[0]", "+I[1]", "+I[2]"));
+
+        String query2 = "select d % 3 as x from t1 where a > 0";
+        runAndCheck(query2, Arrays.asList("+I[1]", "+I[0]", "+I[2]"));
+    }
+
+    @Test
+    void testLargerThen() {
+        String query1 = "select a > 1 as x from t1 where a > 0";
+        runAndCheck(query1, Arrays.asList("+I[false]", "+I[true]", 
"+I[true]"));
+
+        String query2 = "select b > 1 as x from t1 where a > 0";
+        runAndCheck(query2, Arrays.asList("+I[false]", "+I[true]", 
"+I[true]"));
+    }
+
+    @Test
+    void testLessThen() {
+        String query1 = "select a < 2 as x from t1 where a > 0";
+        runAndCheck(query1, Arrays.asList("+I[true]", "+I[false]", 
"+I[false]"));
+
+        String query2 = "select b < 2 as x from t1 where a > 0";
+        runAndCheck(query2, Arrays.asList("+I[true]", "+I[false]", 
"+I[false]"));
+    }
+
+    @Test
+    void testEqual() {
+        String query1 = "select a = 1 as x from t1 where a > 0";
+        runAndCheck(query1, Arrays.asList("+I[true]", "+I[false]", 
"+I[false]"));
+
+        String query2 = "select b = 1 as x from t1 where a > 0";
+        runAndCheck(query2, Arrays.asList("+I[true]", "+I[false]", 
"+I[false]"));
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to