dawidwys commented on code in PR #23761:
URL: https://github.com/apache/flink/pull/23761#discussion_r1407464091


##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/LegacyUserDefinedFunctionInference.java:
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.flink.table.functions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.ArgumentCount;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.ConstantArgumentCount;
+import org.apache.flink.table.types.inference.InputTypeStrategy;
+import org.apache.flink.table.types.inference.Signature;
+import org.apache.flink.table.types.inference.TypeStrategy;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+
+import org.apache.flink.shaded.guava31.com.google.common.primitives.Primitives;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.math.BigDecimal;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeUtils.toInternalConversionClass;
+import static 
org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType;
+
+/**
+ * Ported {@code UserDefinedFunctionUtils} to run some of the type inference 
for legacy functions in
+ * Table API.
+ */
+@Internal
+@Deprecated
+public class LegacyUserDefinedFunctionInference {
+
+    public static InputTypeStrategy 
getInputTypeStrategy(ImperativeAggregateFunction<?, ?> func) {
+        return new InputTypeStrategy() {
+            @Override
+            public ArgumentCount getArgumentCount() {
+                return ConstantArgumentCount.any();
+            }
+
+            @Override
+            public Optional<List<DataType>> inferInputTypes(
+                    CallContext callContext, boolean throwOnFailure) {
+                final List<DataType> argumentDataTypes = 
callContext.getArgumentDataTypes();
+                final DataType accType = getAccumulatorType(func);
+                final LogicalType[] input =
+                        Stream.concat(Stream.of(accType), 
argumentDataTypes.stream())
+                                .map(DataType::getLogicalType)
+                                .toArray(LogicalType[]::new);
+
+                final Optional<Optional<Method>> foundMethod =
+                        Stream.of(
+                                        logicalTypesToInternalClasses(input),
+                                        logicalTypesToExternalClasses(input))
+                                .map(
+                                        signature ->
+                                                getUserDefinedMethod(
+                                                        func,
+                                                        "accumulate",
+                                                        signature,
+                                                        input,
+                                                        cls ->
+                                                                Stream.concat(
+                                                                               
 Stream.of(accType),
+                                                                               
 Arrays.stream(cls)
+                                                                               
         .skip(1)
+                                                                               
         .map(
+                                                                               
                 c ->
+                                                                               
                         fromLegacyInfoToDataType(
+                                                                               
                                 TypeExtractor
+                                                                               
                                         .createTypeInfo(
+                                                                               
                                                 c))))
+                                                                        
.toArray(DataType[]::new)))
+                                .filter(Optional::isPresent)
+                                .findAny();
+
+                if (foundMethod.isPresent()) {
+                    return Optional.of(argumentDataTypes);
+                } else {
+                    return callContext.fail(throwOnFailure, "");
+                }
+            }
+
+            @Override
+            public List<Signature> getExpectedSignatures(FunctionDefinition 
definition) {
+                return getSignatures(func, "accumulate");
+            }
+        };
+    }
+
+    public static InputTypeStrategy getInputTypeStrategy(TableFunction<?> 
func) {
+        return new InputTypeStrategy() {
+            @Override
+            public ArgumentCount getArgumentCount() {
+                return ConstantArgumentCount.any();
+            }
+
+            @Override
+            public Optional<List<DataType>> inferInputTypes(
+                    CallContext callContext, boolean throwOnFailure) {
+                final List<DataType> argumentDataTypes = 
callContext.getArgumentDataTypes();
+                final LogicalType[] input =
+                        argumentDataTypes.stream()
+                                .map(DataType::getLogicalType)
+                                .toArray(LogicalType[]::new);
+
+                final Optional<Method> foundMethod =
+                        getUserDefinedMethod(
+                                func,
+                                "eval",
+                                argumentDataTypes.stream()
+                                        .map(DataType::getConversionClass)
+                                        .toArray(Class<?>[]::new),
+                                input,
+                                cls ->
+                                        Arrays.stream(cls)
+                                                .map(
+                                                        c ->
+                                                                
fromLegacyInfoToDataType(
+                                                                        
TypeExtractor
+                                                                               
 .createTypeInfo(c)))
+                                                .toArray(DataType[]::new));
+
+                if (foundMethod.isPresent()) {
+                    return Optional.of(argumentDataTypes);
+                } else {
+                    return callContext.fail(
+                            throwOnFailure, "Given parameters do not match any 
signature.");
+                }
+            }
+
+            @Override
+            public List<Signature> getExpectedSignatures(FunctionDefinition 
definition) {
+                return getSignatures(func, "eval");
+            }
+        };
+    }
+
+    public static TypeStrategy getOutputTypeStrategy(ScalarFunction func) {
+        return callContext -> {
+            final LogicalType[] params =
+                    callContext.getArgumentDataTypes().stream()
+                            .map(DataType::getLogicalType)
+                            .toArray(LogicalType[]::new);
+            Optional<Class<?>[]> evalParams = getEvalMethodSignature(func, 
params);
+            if (!evalParams.isPresent()) {
+                return Optional.empty();
+            }
+
+            final TypeInformation<?> userDefinedTypeInfo = 
func.getResultType(evalParams.get());
+            if (userDefinedTypeInfo != null) {
+                return 
Optional.of(fromLegacyInfoToDataType(userDefinedTypeInfo));
+            } else {
+                final Optional<Method> eval =
+                        getUserDefinedMethod(
+                                func,
+                                "eval",
+                                logicalTypesToExternalClasses(params),
+                                params,
+                                (paraClasses) ->
+                                        
Arrays.stream(func.getParameterTypes(paraClasses))
+                                                
.map(TypeConversions::fromLegacyInfoToDataType)
+                                                .toArray(DataType[]::new));
+                return eval.flatMap(m -> 
TypeConversions.fromClassToDataType(m.getReturnType()));
+            }
+        };
+    }
+
+    public static InputTypeStrategy getInputTypeStrategy(ScalarFunction func) {
+        return new InputTypeStrategy() {
+            @Override
+            public ArgumentCount getArgumentCount() {
+                return ConstantArgumentCount.any();
+            }
+
+            @Override
+            public Optional<List<DataType>> inferInputTypes(
+                    CallContext callContext, boolean throwOnFailure) {
+                final List<DataType> argumentDataTypes = 
callContext.getArgumentDataTypes();
+                final LogicalType[] input =
+                        argumentDataTypes.stream()
+                                .map(DataType::getLogicalType)
+                                .toArray(LogicalType[]::new);
+
+                final Optional<Method> foundMethod =
+                        getUserDefinedMethod(
+                                func,
+                                "eval",
+                                logicalTypesToExternalClasses(input),
+                                input,
+                                (paraClasses) ->
+                                        
Arrays.stream(func.getParameterTypes(paraClasses))
+                                                
.map(TypeConversions::fromLegacyInfoToDataType)
+                                                .toArray(DataType[]::new));
+
+                if (foundMethod.isPresent()) {
+                    return Optional.of(argumentDataTypes);
+                } else {
+                    return callContext.fail(
+                            throwOnFailure, "Given parameters do not match any 
signature.");
+                }
+            }
+
+            @Override
+            public List<Signature> getExpectedSignatures(FunctionDefinition 
definition) {
+                return getSignatures(func, "eval");
+            }
+        };
+    }
+
+    private static Optional<Class<?>[]> getEvalMethodSignature(
+            ScalarFunction func, LogicalType[] expectedTypes) {
+        return getUserDefinedMethod(
+                        func,
+                        "eval",
+                        logicalTypesToExternalClasses(expectedTypes),
+                        expectedTypes,
+                        (paraClasses) ->
+                                
Arrays.stream(func.getParameterTypes(paraClasses))
+                                        
.map(TypeConversions::fromLegacyInfoToDataType)
+                                        .toArray(DataType[]::new))
+                .map(
+                        m ->
+                                getParamClassesConsiderVarArgs(
+                                        m.isVarArgs(),
+                                        m.getParameterTypes(),
+                                        expectedTypes.length));
+    }
+
+    private static Class<?>[] getParamClassesConsiderVarArgs(
+            boolean isVarArgs, Class<?>[] matchingSignature, int 
expectedLength) {
+        return IntStream.range(0, expectedLength)

Review Comment:
   Not sure if that's a better approach. 
   
   To be honest I like the single chain approach a bit more. What you're 
suggesting would look:
   ```
   final Class<?>[] array = new Class<?>[types.length];
   Arrays.setAll(array, i -> {
           final LogicalType t = types[i];
           if (t == null) {
               return null;
           } else {
               return TypeConversions.fromLogicalToDataType(t)
                       .getConversionClass();
           }
       });
   return array;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to