From cbe45f4d5ba0132e0a958fc89a0ce90114c8a7fd Mon Sep 17 00:00:00 2001 From: Alan Sheinberg Date: Tue, 27 May 2025 16:11:52 -0700 Subject: [PATCH] [FLINK-37724] Adds AsyncTableFunction as a fully supported UDF type --- .../execution_config_configuration.html | 32 ++- .../java/typeutils/TypeExtractionUtils.java | 25 ++ .../typeutils/TypeExtractionUtilsTest.java | 74 +++++ .../api/config/ExecutionConfigOptions.java | 45 +++- .../functions/UserDefinedFunctionHelper.java | 42 ++- .../extraction/FunctionMappingExtractor.java | 19 +- .../extraction/TypeInferenceExtractor.java | 6 +- .../types/inference/SystemTypeInference.java | 4 +- .../UserDefinedFunctionHelperTest.java | 69 +++++ .../TypeInferenceExtractorTest.java | 159 +++++++++++ .../catalog/FunctionCatalogOperatorTable.java | 4 +- .../codegen/AsyncCorrelateCodeGenerator.java | 106 ++++++++ .../bridging/BridgingSqlFunction.java | 5 +- .../exec/common/CommonExecAsyncCalc.java | 2 +- .../exec/common/CommonExecAsyncCorrelate.java | 154 +++++++++++ .../exec/stream/StreamExecAsyncCorrelate.java | 93 +++++++ .../FlinkLogicalTableFunctionScan.java | 1 + .../stream/StreamPhysicalAsyncCorrelate.java | 95 +++++++ .../rules/logical/AsyncCalcSplitRule.java | 26 +- .../logical/AsyncCorrelateSplitRule.java | 12 +- .../logical/RemoteCorrelateSplitRule.java | 2 +- .../StreamPhysicalAsyncCorrelateRule.java | 150 +++++++++++ ...PhysicalConstantTableFunctionScanRule.java | 14 + .../table/planner/plan/utils/AsyncUtil.java | 66 ++++- .../plan/utils/ExecNodeMetadataUtil.java | 2 + .../calls/BridgingFunctionGenUtil.scala | 40 +-- .../plan/rules/FlinkStreamRuleSets.scala | 10 +- .../stream/StreamPhysicalCorrelateRule.scala | 12 +- .../AsyncCorrelateCodeGeneratorTest.java | 245 +++++++++++++++++ .../stream/AsyncCorrelateRestoreTest.java | 43 +++ .../stream/AsyncCorrelateTestPrograms.java | 238 ++++++++++++++++ .../rules/logical/AsyncCalcSplitRuleTest.java | 6 + .../logical/AsyncCorrelateSplitRuleTest.java | 42 +++ .../stream/table/AsyncCorrelateITCase.java | 255 ++++++++++++++++++ .../utils/JavaUserDefinedTableFunctions.java | 77 ++++++ .../rules/logical/AsyncCalcSplitRuleTest.xml | 18 ++ .../logical/AsyncCorrelateSplitRuleTest.xml | 69 ++++- .../plan/async-correlate-catalog-func.json | 140 ++++++++++ .../savepoint/_metadata | Bin 0 -> 9035 bytes .../plan/async-correlate-exception.json | 140 ++++++++++ .../savepoint/_metadata | Bin 0 -> 8896 bytes .../plan/async-correlate-join-filter.json | 207 ++++++++++++++ .../savepoint/_metadata | Bin 0 -> 8907 bytes .../plan/async-correlate-left-join.json | 136 ++++++++++ .../savepoint/_metadata | Bin 0 -> 9035 bytes .../plan/async-correlate-system-func.json | 140 ++++++++++ .../savepoint/_metadata | Bin 0 -> 9035 bytes .../plan/stream/sql/join/LookupJoinTest.scala | 31 +-- .../planner/plan/utils/lookupFunctions.scala | 7 +- .../calc/async/AsyncFunctionRunner.java | 2 +- .../async/DelegatingAsyncResultFuture.java | 3 +- .../correlate/async/AsyncCorrelateRunner.java | 159 +++++++++++ .../DelegatingAsyncTableResultFuture.java} | 37 ++- .../correlate/AsyncCorrelateRunnerTest.java | 195 ++++++++++++++ 54 files changed, 3371 insertions(+), 88 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractionUtilsTest.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/codegen/AsyncCorrelateCodeGenerator.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecAsyncCorrelate.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAsyncCorrelate.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalAsyncCorrelate.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalAsyncCorrelateRule.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/codegen/AsyncCorrelateCodeGeneratorTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/AsyncCorrelateRestoreTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/AsyncCorrelateTestPrograms.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/AsyncCorrelateITCase.java create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-catalog-func/plan/async-correlate-catalog-func.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-catalog-func/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-exception/plan/async-correlate-exception.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-exception/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-join-filter/plan/async-correlate-join-filter.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-join-filter/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-left-join/plan/async-correlate-left-join.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-left-join/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-system-func/plan/async-correlate-system-func.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-system-func/savepoint/_metadata create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/AsyncCorrelateRunner.java rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/{join/lookup/DelegatingResultFuture.java => correlate/async/DelegatingAsyncTableResultFuture.java} (55%) create mode 100644 flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/correlate/AsyncCorrelateRunnerTest.java diff --git a/docs/layouts/shortcodes/generated/execution_config_configuration.html b/docs/layouts/shortcodes/generated/execution_config_configuration.html index 770f2f5b52499..aa768ad73dc79 100644 --- a/docs/layouts/shortcodes/generated/execution_config_configuration.html +++ b/docs/layouts/shortcodes/generated/execution_config_configuration.html @@ -30,7 +30,7 @@
table.exec.async-scalar.buffer-capacity

Streaming 10 Integer - The max number of async i/o operation that the async lookup join can trigger. + The max number of async i/o operation that the async scalar function can trigger.
table.exec.async-scalar.max-attempts

Streaming @@ -62,6 +62,36 @@ Boolean Set whether to use the SQL/Table operators based on the asynchronous state api. Default value is false. + +
table.exec.async-table.buffer-capacity

Streaming + 10 + Integer + The max number of async i/o operations that the async table function can trigger. + + +
table.exec.async-table.max-attempts

Streaming + 3 + Integer + The max number of async retry attempts to make before task execution is failed. + + +
table.exec.async-table.retry-delay

Streaming + 100 ms + Duration + The delay to wait before trying again. + + +
table.exec.async-table.retry-strategy

Streaming + FIXED_DELAY +

Enum

+ Restart strategy which will be used, FIXED_DELAY by default.

Possible values: + + +
table.exec.async-table.timeout

Streaming + 3 min + Duration + The async timeout for the asynchronous operation to complete. +
table.exec.deduplicate.insert-update-after-sensitive-enabled

Streaming true diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java index 907508c166234..f026655c55d39 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java @@ -34,6 +34,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import static org.apache.flink.shaded.asm9.org.objectweb.asm.Type.getConstructorDescriptor; import static org.apache.flink.shaded.asm9.org.objectweb.asm.Type.getMethodDescriptor; @@ -373,4 +374,28 @@ public static void validateLambdaType(Class baseClass, Type t) { + "Otherwise the type has to be specified explicitly using type information."); } } + + /** + * Will return true if the type of the given generic class type matches clazz. + * + * @param clazz The generic class to check against + * @param type The type to be checked + */ + public static boolean isGenericOfClass(Class clazz, Type type) { + Optional parameterized = getParameterizedType(type); + return clazz.equals(type) + || parameterized.isPresent() && clazz.equals(parameterized.get().getRawType()); + } + + /** + * Returns an optional of a ParameterizedType, if that's what the type is. + * + * @param type The type to check + * @return optional which is present if the type is a ParameterizedType + */ + public static Optional getParameterizedType(Type type) { + return Optional.of(type) + .filter(p -> p instanceof ParameterizedType) + .map(ParameterizedType.class::cast); + } } diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractionUtilsTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractionUtilsTest.java new file mode 100644 index 0000000000000..9dfa3b572ac3c --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractionUtilsTest.java @@ -0,0 +1,74 @@ +/* + * 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.api.java.typeutils; + +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Method; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.List; +import java.util.Optional; + +import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.getAllDeclaredMethods; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link TypeExtractionUtils}. */ +@SuppressWarnings("rawtypes") +public class TypeExtractionUtilsTest { + + @Test + void testIsGeneric() throws Exception { + Method method = getMethod(IsGeneric.class, "m1"); + Type firstParam = method.getGenericParameterTypes()[0]; + assertThat(TypeExtractionUtils.isGenericOfClass(List.class, firstParam)).isTrue(); + + method = getMethod(IsGeneric.class, "m2"); + firstParam = method.getGenericParameterTypes()[0]; + assertThat(TypeExtractionUtils.isGenericOfClass(List.class, firstParam)).isTrue(); + } + + @Test + void testGetParameterizedType() throws Exception { + Method method = getMethod(IsGeneric.class, "m1"); + Type firstParam = method.getGenericParameterTypes()[0]; + Optional parameterizedType = + TypeExtractionUtils.getParameterizedType(firstParam); + assertThat(parameterizedType).isPresent(); + assertThat(parameterizedType.get().getRawType()).isEqualTo(List.class); + assertThat(parameterizedType.get().getActualTypeArguments()[0]).isEqualTo(Integer.class); + + method = getMethod(IsGeneric.class, "m2"); + firstParam = method.getGenericParameterTypes()[0]; + assertThat(TypeExtractionUtils.getParameterizedType(firstParam)).isEmpty(); + } + + private Method getMethod(Class clazz, String name) throws Exception { + return getAllDeclaredMethods(clazz).stream() + .filter(m -> m.getName().equals(name)) + .findFirst() + .orElseThrow(); + } + + public static class IsGeneric { + public void m1(List list) {} + + public void m2(List list) {} + } +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java index a1847e93b9e77..4a72d48c37a76 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java @@ -409,7 +409,7 @@ public class ExecutionConfigOptions { .intType() .defaultValue(10) .withDescription( - "The max number of async i/o operation that the async lookup join can trigger."); + "The max number of async i/o operation that the async scalar function can trigger."); @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) public static final ConfigOption TABLE_EXEC_ASYNC_SCALAR_TIMEOUT = @@ -443,6 +443,49 @@ public class ExecutionConfigOptions { "The max number of async retry attempts to make before task " + "execution is failed."); + // ------------------------------------------------------------------------ + // Async Table Function + // ------------------------------------------------------------------------ + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) + public static final ConfigOption TABLE_EXEC_ASYNC_TABLE_BUFFER_CAPACITY = + key("table.exec.async-table.buffer-capacity") + .intType() + .defaultValue(10) + .withDescription( + "The max number of async i/o operations that the async table function can trigger."); + + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) + public static final ConfigOption TABLE_EXEC_ASYNC_TABLE_TIMEOUT = + key("table.exec.async-table.timeout") + .durationType() + .defaultValue(Duration.ofMinutes(3)) + .withDescription( + "The async timeout for the asynchronous operation to complete, including any retries which may occur."); + + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) + public static final ConfigOption TABLE_EXEC_ASYNC_TABLE_RETRY_STRATEGY = + key("table.exec.async-table.retry-strategy") + .enumType(RetryStrategy.class) + .defaultValue(RetryStrategy.FIXED_DELAY) + .withDescription( + "Restart strategy which will be used, FIXED_DELAY by default."); + + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) + public static final ConfigOption TABLE_EXEC_ASYNC_TABLE_RETRY_DELAY = + key("table.exec.async-table.retry-delay") + .durationType() + .defaultValue(Duration.ofMillis(100)) + .withDescription("The delay to wait before trying again."); + + @Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) + public static final ConfigOption TABLE_EXEC_ASYNC_TABLE_MAX_ATTEMPTS = + key("table.exec.async-table.max-attempts") + .intType() + .defaultValue(3) + .withDescription( + "The max number of async retry attempts to make before task " + + "execution is failed."); + // ------------------------------------------------------------------------ // MiniBatch Options // ------------------------------------------------------------------------ diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/UserDefinedFunctionHelper.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/UserDefinedFunctionHelper.java index 25d7e4ea7cfb7..6f402a619983f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/UserDefinedFunctionHelper.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/UserDefinedFunctionHelper.java @@ -48,13 +48,17 @@ import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; import java.util.Arrays; +import java.util.Collection; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.getAllDeclaredMethods; +import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.getParameterizedType; +import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.isGenericOfClass; import static org.apache.flink.util.Preconditions.checkState; /** @@ -477,11 +481,12 @@ private static void validateImplementationMethods( validateImplementationMethod(functionClass, false, false, SCALAR_EVAL); } else if (AsyncScalarFunction.class.isAssignableFrom(functionClass)) { validateImplementationMethod(functionClass, false, false, ASYNC_SCALAR_EVAL); - validateAsyncImplementationMethod(functionClass, ASYNC_SCALAR_EVAL); + validateAsyncImplementationMethod(functionClass, false, ASYNC_SCALAR_EVAL); } else if (TableFunction.class.isAssignableFrom(functionClass)) { validateImplementationMethod(functionClass, true, false, TABLE_EVAL); } else if (AsyncTableFunction.class.isAssignableFrom(functionClass)) { validateImplementationMethod(functionClass, true, false, ASYNC_TABLE_EVAL); + validateAsyncImplementationMethod(functionClass, true, ASYNC_TABLE_EVAL); } else if (AggregateFunction.class.isAssignableFrom(functionClass)) { validateImplementationMethod(functionClass, true, false, AGGREGATE_ACCUMULATE); validateImplementationMethod(functionClass, true, true, AGGREGATE_RETRACT); @@ -541,7 +546,9 @@ private static void validateImplementationMethod( } private static void validateAsyncImplementationMethod( - Class clazz, String... methodNameOptions) { + Class clazz, + boolean verifyFutureContainsCollection, + String... methodNameOptions) { final Set nameSet = new HashSet<>(Arrays.asList(methodNameOptions)); final List methods = getAllDeclaredMethods(clazz); for (Method method : methods) { @@ -558,18 +565,31 @@ private static void validateAsyncImplementationMethod( if (method.getParameterCount() >= 1) { Type firstParam = method.getGenericParameterTypes()[0]; firstParam = ExtractionUtils.resolveVariableWithClassContext(clazz, firstParam); - if (CompletableFuture.class.equals(firstParam) - || firstParam instanceof ParameterizedType - && CompletableFuture.class.equals( - ((ParameterizedType) firstParam).getRawType())) { - foundParam = true; + if (isGenericOfClass(CompletableFuture.class, firstParam)) { + Optional parameterized = getParameterizedType(firstParam); + if (!verifyFutureContainsCollection) { + foundParam = true; + } else if (parameterized.isPresent() + && parameterized.get().getActualTypeArguments().length > 0) { + firstParam = parameterized.get().getActualTypeArguments()[0]; + if (isGenericOfClass(Collection.class, firstParam)) { + foundParam = true; + } + } } } if (!foundParam) { - throw new ValidationException( - String.format( - "Method '%s' of function class '%s' must have a first argument of type java.util.concurrent.CompletableFuture.", - method.getName(), clazz.getName())); + if (!verifyFutureContainsCollection) { + throw new ValidationException( + String.format( + "Method '%s' of function class '%s' must have a first argument of type java.util.concurrent.CompletableFuture.", + method.getName(), clazz.getName())); + } else { + throw new ValidationException( + String.format( + "Method '%s' of function class '%s' must have a first argument of type java.util.concurrent.CompletableFuture.", + method.getName(), clazz.getName())); + } } } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/FunctionMappingExtractor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/FunctionMappingExtractor.java index 69279365c4651..e29bebe425d75 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/FunctionMappingExtractor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/FunctionMappingExtractor.java @@ -43,6 +43,7 @@ import java.util.concurrent.CompletableFuture; import java.util.stream.Stream; +import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.getParameterizedType; import static org.apache.flink.table.types.extraction.ExtractionUtils.collectAnnotationsOfClass; import static org.apache.flink.table.types.extraction.ExtractionUtils.collectAnnotationsOfMethod; import static org.apache.flink.table.types.extraction.ExtractionUtils.extractionError; @@ -208,7 +209,8 @@ static MethodVerification createParameterVerification(boolean requireAccumulator * Verification that checks a method by parameters (arguments only) with mandatory {@link * CompletableFuture}. */ - static MethodVerification createParameterAndCompletableFutureVerification(Class baseClass) { + static MethodVerification createParameterAndCompletableFutureVerification( + Class baseClass, @Nullable Class nestedArgumentClass) { return (method, state, arguments, result) -> { checkNoState(state); checkScalarArgumentsOnly(arguments); @@ -220,11 +222,24 @@ static MethodVerification createParameterAndCompletableFutureVerification(Class< final Class resultClass = result.toClass(); Type genericType = method.getGenericParameterTypes()[0]; genericType = resolveVariableWithClassContext(baseClass, genericType); - if (!(genericType instanceof ParameterizedType)) { + Optional parameterized = getParameterizedType(genericType); + if (!parameterized.isPresent()) { throw extractionError( "The method '%s' needs generic parameters for the CompletableFuture at position %d.", method.getName(), 0); } + // If nestedArgumentClass is given, it is assumed to be a generic parameters of + // argumentClass, also at the position genericPos + if (nestedArgumentClass != null) { + genericType = parameterized.get().getActualTypeArguments()[0]; + parameterized = getParameterizedType(genericType); + if (!parameterized.isPresent() + || !parameterized.get().getRawType().equals(nestedArgumentClass)) { + throw extractionError( + "The method '%s' expects nested generic type CompletableFuture<%s> for the %d arg.", + method.getName(), nestedArgumentClass.getName(), 0); + } + } final Type returnType = ((ParameterizedType) genericType).getActualTypeArguments()[0]; Class returnTypeClass = getClassFromType(returnType); // Parameters should be validated using strict autoboxing. diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/TypeInferenceExtractor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/TypeInferenceExtractor.java index 19307d06fae76..5c3f2b627b9ba 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/TypeInferenceExtractor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/TypeInferenceExtractor.java @@ -45,6 +45,7 @@ import javax.annotation.Nullable; +import java.util.Collection; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -106,7 +107,7 @@ public static TypeInference forAsyncScalarFunction( null, null, createOutputFromGenericInMethod(0, 0, true), - createParameterAndCompletableFutureVerification(function)); + createParameterAndCompletableFutureVerification(function, null)); return extractTypeInference(mappingExtractor, false); } @@ -172,7 +173,8 @@ public static TypeInference forAsyncTableFunction( null, null, createOutputFromGenericInClass(AsyncTableFunction.class, 0, true), - createParameterAndCompletableFutureVerification(function)); + createParameterAndCompletableFutureVerification( + function, Collection.class)); return extractTypeInference(mappingExtractor, false); } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/SystemTypeInference.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/SystemTypeInference.java index 0dbdc20ae82d0..8b9e7b8d25450 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/SystemTypeInference.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/SystemTypeInference.java @@ -198,7 +198,9 @@ private static TypeStrategy deriveSystemOutputStrategy( FunctionKind functionKind, @Nullable List staticArgs, TypeStrategy outputStrategy) { - if (functionKind != FunctionKind.TABLE && functionKind != FunctionKind.PROCESS_TABLE) { + if (functionKind != FunctionKind.TABLE + && functionKind != FunctionKind.PROCESS_TABLE + && functionKind != FunctionKind.ASYNC_TABLE) { return outputStrategy; } return new SystemOutputStrategy(functionKind, staticArgs, outputStrategy); diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/functions/UserDefinedFunctionHelperTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/functions/UserDefinedFunctionHelperTest.java index 2aa2d3a938b25..71cb771065a9a 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/functions/UserDefinedFunctionHelperTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/functions/UserDefinedFunctionHelperTest.java @@ -32,6 +32,7 @@ import javax.annotation.Nullable; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Optional; @@ -135,32 +136,63 @@ private static List testSpecs() { + PrivateMethodScalarFunction.class.getName() + "' is not public."), TestSpec.forClass(ValidAsyncScalarFunction.class).expectSuccess(), + TestSpec.forClass(ValidAsyncTableFunction.class).expectSuccess(), TestSpec.forInstance(new ValidAsyncScalarFunction()).expectSuccess(), TestSpec.forClass(PrivateAsyncScalarFunction.class) .expectErrorMessage( "Function class '" + PrivateAsyncScalarFunction.class.getName() + "' is not public."), + TestSpec.forClass(PrivateAsyncTableFunction.class) + .expectErrorMessage( + "Function class '" + + PrivateAsyncTableFunction.class.getName() + + "' is not public."), TestSpec.forClass(MissingImplementationAsyncScalarFunction.class) .expectErrorMessage( "Function class '" + MissingImplementationAsyncScalarFunction.class.getName() + "' does not implement a method named 'eval'."), + TestSpec.forClass(MissingImplementationAsyncTableFunction.class) + .expectErrorMessage( + "Function class '" + + MissingImplementationAsyncTableFunction.class.getName() + + "' does not implement a method named 'eval'."), TestSpec.forClass(PrivateMethodAsyncScalarFunction.class) .expectErrorMessage( "Method 'eval' of function class '" + PrivateMethodAsyncScalarFunction.class.getName() + "' is not public."), + TestSpec.forClass(PrivateMethodAsyncTableFunction.class) + .expectErrorMessage( + "Method 'eval' of function class '" + + PrivateMethodAsyncTableFunction.class.getName() + + "' is not public."), TestSpec.forClass(NonVoidAsyncScalarFunction.class) .expectErrorMessage( "Method 'eval' of function class '" + NonVoidAsyncScalarFunction.class.getName() + "' must be void."), + TestSpec.forClass(NonVoidAsyncTableFunction.class) + .expectErrorMessage( + "Method 'eval' of function class '" + + NonVoidAsyncTableFunction.class.getName() + + "' must be void."), TestSpec.forClass(NoFutureAsyncScalarFunction.class) .expectErrorMessage( "Method 'eval' of function class '" + NoFutureAsyncScalarFunction.class.getName() + "' must have a first argument of type java.util.concurrent.CompletableFuture."), + TestSpec.forClass(NoFutureAsyncTableFunction.class) + .expectErrorMessage( + "Method 'eval' of function class '" + + NoFutureAsyncTableFunction.class.getName() + + "' must have a first argument of type java.util.concurrent.CompletableFuture."), + TestSpec.forClass(NoFutureAsyncTableFunction2.class) + .expectErrorMessage( + "Method 'eval' of function class '" + + NoFutureAsyncTableFunction2.class.getName() + + "' must have a first argument of type java.util.concurrent.CompletableFuture."), TestSpec.forInstance(new ValidTableAggregateFunction()).expectSuccess(), TestSpec.forInstance(new MissingEmitTableAggregateFunction()) .expectErrorMessage( @@ -295,20 +327,40 @@ public static class ValidAsyncScalarFunction extends AsyncScalarFunction { public void eval(CompletableFuture future, int i) {} } + /** Valid table function. */ + public static class ValidAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, int i) {} + } + private static class PrivateAsyncScalarFunction extends AsyncScalarFunction { public void eval(CompletableFuture future, int i) {} } + private static class PrivateAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, int i) {} + } + /** No implementation method. */ public static class MissingImplementationAsyncScalarFunction extends AsyncScalarFunction { // nothing to do } + /** No implementation method. */ + public static class MissingImplementationAsyncTableFunction + extends AsyncTableFunction { + // nothing to do + } + /** Implementation method is private. */ public static class PrivateMethodAsyncScalarFunction extends AsyncScalarFunction { private void eval(CompletableFuture future, int i) {} } + /** Implementation method is private. */ + public static class PrivateMethodAsyncTableFunction extends AsyncTableFunction { + private void eval(CompletableFuture> future, int i) {} + } + /** Implementation method isn't void. */ public static class NonVoidAsyncScalarFunction extends AsyncScalarFunction { public String eval(CompletableFuture future, int i) { @@ -317,10 +369,27 @@ public String eval(CompletableFuture future, int i) { } /** Implementation method isn't void. */ + public static class NonVoidAsyncTableFunction extends AsyncScalarFunction { + public String eval(CompletableFuture> future, int i) { + return ""; + } + } + + /** First param isn't a future. */ public static class NoFutureAsyncScalarFunction extends AsyncScalarFunction { public void eval(int i) {} } + /** First param isn't a future. */ + public static class NoFutureAsyncTableFunction extends AsyncTableFunction { + public void eval(int i) {} + } + + /** First param is a future, but with the wrong contained type. */ + public static class NoFutureAsyncTableFunction2 extends AsyncTableFunction { + public void eval(CompletableFuture> future, int i) {} + } + /** Valid table aggregate function. */ public static class ValidTableAggregateFunction extends TableAggregateFunction { diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java index e1d6d90115a66..67f643dacdc56 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java @@ -31,6 +31,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.AsyncScalarFunction; +import org.apache.flink.table.functions.AsyncTableFunction; import org.apache.flink.table.functions.ProcessTableFunction; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableAggregateFunction; @@ -58,6 +59,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.EnumSet; import java.util.LinkedHashMap; import java.util.List; @@ -179,6 +181,11 @@ private static Stream functionSpecs() { .expectEmptyStaticArguments() .expectOutput(TypeStrategies.explicit(DataTypes.INT())), // --- + // no arguments async table + TestSpec.forAsyncTableFunction(ZeroArgFunctionAsyncTable.class) + .expectEmptyStaticArguments() + .expectOutput(TypeStrategies.explicit(DataTypes.INT())), + // --- // test primitive arguments extraction TestSpec.forScalarFunction(MixedArgFunction.class) .expectStaticArgument( @@ -195,6 +202,14 @@ private static Stream functionSpecs() { .expectStaticArgument(StaticArgument.scalar("d", DataTypes.DOUBLE(), false)) .expectOutput(TypeStrategies.explicit(DataTypes.INT())), // --- + // test primitive arguments extraction async table + TestSpec.forAsyncTableFunction(MixedArgFunctionAsyncTable.class) + .expectStaticArgument( + StaticArgument.scalar( + "i", DataTypes.INT().notNull().bridgedTo(int.class), false)) + .expectStaticArgument(StaticArgument.scalar("d", DataTypes.DOUBLE(), false)) + .expectOutput(TypeStrategies.explicit(DataTypes.INT())), + // --- // test overloaded arguments extraction TestSpec.forScalarFunction(OverloadedFunction.class) .expectOutputMapping( @@ -234,6 +249,25 @@ private static Stream functionSpecs() { }), TypeStrategies.explicit(DataTypes.BIGINT())), // --- + // test overloaded arguments extraction async + TestSpec.forAsyncTableFunction(OverloadedFunctionAsyncTable.class) + .expectOutputMapping( + InputTypeStrategies.sequence( + new String[] {"i", "d"}, + new ArgumentTypeStrategy[] { + InputTypeStrategies.explicit( + DataTypes.INT().notNull().bridgedTo(int.class)), + InputTypeStrategies.explicit(DataTypes.DOUBLE()) + }), + TypeStrategies.explicit(DataTypes.INT())) + .expectOutputMapping( + InputTypeStrategies.sequence( + new String[] {"s"}, + new ArgumentTypeStrategy[] { + InputTypeStrategies.explicit(DataTypes.STRING()) + }), + TypeStrategies.explicit(DataTypes.INT())), + // --- // test varying arguments extraction TestSpec.forScalarFunction(VarArgFunction.class) .expectOutputMapping( @@ -260,6 +294,19 @@ private static Stream functionSpecs() { }), TypeStrategies.explicit(DataTypes.STRING())), // --- + // test varying arguments extraction async table + TestSpec.forAsyncTableFunction(VarArgFunctionAsyncTable.class) + .expectOutputMapping( + InputTypeStrategies.varyingSequence( + new String[] {"i", "more"}, + new ArgumentTypeStrategy[] { + InputTypeStrategies.explicit( + DataTypes.INT().notNull().bridgedTo(int.class)), + InputTypeStrategies.explicit( + DataTypes.INT().notNull().bridgedTo(int.class)) + }), + TypeStrategies.explicit(DataTypes.STRING())), + // --- // test varying arguments extraction with byte TestSpec.forScalarFunction(VarArgWithByteFunction.class) .expectOutputMapping( @@ -286,6 +333,19 @@ private static Stream functionSpecs() { }), TypeStrategies.explicit(DataTypes.STRING())), // --- + // test varying arguments extraction with byte async + TestSpec.forAsyncTableFunction(VarArgWithByteFunctionAsyncTable.class) + .expectOutputMapping( + InputTypeStrategies.varyingSequence( + new String[] {"bytes"}, + new ArgumentTypeStrategy[] { + InputTypeStrategies.explicit( + DataTypes.TINYINT() + .notNull() + .bridgedTo(byte.class)) + }), + TypeStrategies.explicit(DataTypes.STRING())), + // --- // output hint with input extraction TestSpec.forScalarFunction(ExtractWithOutputHintFunction.class) .expectStaticArgument(StaticArgument.scalar("i", DataTypes.INT(), false)) @@ -296,6 +356,11 @@ private static Stream functionSpecs() { .expectStaticArgument(StaticArgument.scalar("i", DataTypes.INT(), false)) .expectOutput(TypeStrategies.explicit(DataTypes.INT())), // --- + // output hint with input extraction + TestSpec.forAsyncTableFunction(ExtractWithOutputHintFunctionAsyncTable.class) + .expectStaticArgument(StaticArgument.scalar("i", DataTypes.INT(), false)) + .expectOutput(TypeStrategies.explicit(DataTypes.INT())), + // --- // output extraction with input hints TestSpec.forScalarFunction(ExtractWithInputHintFunction.class) .expectStaticArgument(StaticArgument.scalar("i", DataTypes.INT(), false)) @@ -395,6 +460,25 @@ private static Stream functionSpecs() { "Considering all hints, the method should comply with the signature:\n" + "eval(java.util.concurrent.CompletableFuture, int[])"), // --- + // mismatch between hints and implementation regarding return type + TestSpec.forAsyncTableFunction(InvalidMethodTableFunctionAsync.class) + .expectErrorMessage( + "Considering all hints, the method should comply with the signature:\n" + + "eval(java.util.concurrent.CompletableFuture, int[])"), + // --- + TestSpec.forAsyncTableFunction(InvalidMethodTableFunctionMissingCollection.class) + .expectErrorMessage( + "The method 'eval' expects nested generic type CompletableFuture for the 0 arg."), + // --- + TestSpec.forAsyncTableFunction(InvalidMethodTableFunctionWrongGeneric.class) + .expectErrorMessage( + "The method 'eval' expects nested generic type CompletableFuture for the 0 arg."), + // --- + TestSpec.forAsyncTableFunction(ConflictingReturnTypesAsyncTable.class) + .expectErrorMessage( + "Considering all hints, the method should comply with the signature:\n" + + "eval(java.util.concurrent.CompletableFuture, int)"), + // --- // mismatch between hints and implementation regarding accumulator TestSpec.forAggregateFunction(InvalidMethodAggregateFunction.class) .expectErrorMessage( @@ -514,6 +598,15 @@ private static Stream functionSpecs() { DataTypes.ROW(DataTypes.FIELD("i", DataTypes.INT())) .bridgedTo(RowData.class))), // --- + TestSpec.forAsyncTableFunction( + "A data type hint on the method is used for enriching (not a function output hint)", + DataTypeHintOnTableFunctionAsync.class) + .expectEmptyStaticArguments() + .expectOutput( + TypeStrategies.explicit( + DataTypes.ROW(DataTypes.FIELD("i", DataTypes.INT())) + .bridgedTo(RowData.class))), + // --- TestSpec.forScalarFunction( "Scalar function with arguments hints", ArgumentHintScalarFunction.class) @@ -1203,6 +1296,21 @@ static TestSpec forAsyncScalarFunction( new DataTypeFactoryMock(), function)); } + @SuppressWarnings("rawtypes") + static TestSpec forAsyncTableFunction(Class> function) { + return forAsyncTableFunction(null, function); + } + + @SuppressWarnings("rawtypes") + static TestSpec forAsyncTableFunction( + String description, Class> function) { + return new TestSpec( + description == null ? function.getSimpleName() : description, + () -> + TypeInferenceExtractor.forAsyncTableFunction( + new DataTypeFactoryMock(), function)); + } + static TestSpec forAggregateFunction(Class> function) { return new TestSpec( function.getSimpleName(), @@ -2046,38 +2154,89 @@ private static class ZeroArgFunctionAsync extends AsyncScalarFunction { public void eval(CompletableFuture f) {} } + private static class ZeroArgFunctionAsyncTable extends AsyncTableFunction { + public void eval(CompletableFuture> f) {} + } + private static class MixedArgFunctionAsync extends AsyncScalarFunction { public void eval(CompletableFuture f, int i, Double d) {} } + private static class MixedArgFunctionAsyncTable extends AsyncTableFunction { + public void eval(CompletableFuture> f, int i, Double d) {} + } + private static class OverloadedFunctionAsync extends AsyncScalarFunction { public void eval(CompletableFuture f, int i, Double d) {} public void eval(CompletableFuture f, String s) {} } + private static class OverloadedFunctionAsyncTable extends AsyncTableFunction { + public void eval(CompletableFuture> f, int i, Double d) {} + + public void eval(CompletableFuture> f, String s) {} + } + + private static class ConflictingReturnTypesAsyncTable extends AsyncTableFunction { + public void eval(CompletableFuture> f, int i) {} + } + private static class VarArgFunctionAsync extends AsyncScalarFunction { public void eval(CompletableFuture f, int i, int... more) {} } + private static class VarArgFunctionAsyncTable extends AsyncTableFunction { + public void eval(CompletableFuture> f, int i, int... more) {} + } + private static class VarArgWithByteFunctionAsync extends AsyncScalarFunction { public void eval(CompletableFuture f, byte... bytes) {} } + private static class VarArgWithByteFunctionAsyncTable extends AsyncTableFunction { + public void eval(CompletableFuture> f, byte... bytes) {} + } + @FunctionHint(output = @DataTypeHint("INT")) private static class ExtractWithOutputHintFunctionAsync extends AsyncScalarFunction { public void eval(CompletableFuture f, Integer i) {} } + @FunctionHint(output = @DataTypeHint("INT")) + private static class ExtractWithOutputHintFunctionAsyncTable + extends AsyncTableFunction { + public void eval(CompletableFuture> f, Integer i) {} + } + @FunctionHint(output = @DataTypeHint("STRING")) private static class InvalidMethodScalarFunctionAsync extends AsyncScalarFunction { public void eval(CompletableFuture f, int[] i) {} } + @FunctionHint(output = @DataTypeHint("STRING")) + private static class InvalidMethodTableFunctionAsync extends AsyncTableFunction { + public void eval(CompletableFuture> f, int[] i) {} + } + + private static class InvalidMethodTableFunctionMissingCollection + extends AsyncTableFunction { + public void eval(CompletableFuture f, int[] i) {} + } + + private static class InvalidMethodTableFunctionWrongGeneric extends AsyncTableFunction { + public void eval(CompletableFuture> f, int[] i) {} + } + private static class DataTypeHintOnScalarFunctionAsync extends AsyncScalarFunction { public void eval(@DataTypeHint("ROW") CompletableFuture f) {} } + private static class DataTypeHintOnTableFunctionAsync extends AsyncTableFunction { + @DataTypeHint(value = "ROW", bridgedTo = RowData.class) + public void eval(CompletableFuture> f) {} + } + private static class ArgumentHintScalarFunction extends ScalarFunction { @FunctionHint( arguments = { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java index 38ac039a26e3a..28f8157f6f07d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java @@ -190,7 +190,9 @@ private boolean verifyFunctionKind( final FunctionKind kind = definition.getKind(); - if (kind == FunctionKind.TABLE || kind == FunctionKind.PROCESS_TABLE) { + if (kind == FunctionKind.TABLE + || kind == FunctionKind.ASYNC_TABLE + || kind == FunctionKind.PROCESS_TABLE) { return true; } else if (kind == FunctionKind.SCALAR || kind == FunctionKind.ASYNC_SCALAR diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/codegen/AsyncCorrelateCodeGenerator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/codegen/AsyncCorrelateCodeGenerator.java new file mode 100644 index 0000000000000..de4647236370c --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/codegen/AsyncCorrelateCodeGenerator.java @@ -0,0 +1,106 @@ +/* + * 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.planner.codegen; + +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.functions.FunctionKind; +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.runtime.generated.GeneratedFunction; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexVisitorImpl; + +import java.util.Optional; + +/** + * Generates an {@link AsyncFunction} which can be used to evaluate correlate invocations from an + * async table function. + */ +public class AsyncCorrelateCodeGenerator { + + public static GeneratedFunction> generateFunction( + String name, + RowType inputType, + RowType returnType, + RexCall invocation, + ReadableConfig tableConfig, + ClassLoader classLoader) { + CodeGeneratorContext ctx = new CodeGeneratorContext(tableConfig, classLoader); + String processCode = + generateProcessCode(ctx, inputType, invocation, CodeGenUtils.DEFAULT_INPUT1_TERM()); + return FunctionCodeGenerator.generateFunction( + ctx, + name, + getFunctionClass(), + processCode, + returnType, + inputType, + CodeGenUtils.DEFAULT_INPUT1_TERM(), + JavaScalaConversionUtil.toScala(Optional.empty()), + JavaScalaConversionUtil.toScala(Optional.empty()), + CodeGenUtils.DEFAULT_COLLECTOR_TERM(), + CodeGenUtils.DEFAULT_CONTEXT_TERM()); + } + + @SuppressWarnings("unchecked") + private static Class> getFunctionClass() { + return (Class>) (Object) AsyncFunction.class; + } + + private static String generateProcessCode( + CodeGeneratorContext ctx, RowType inputType, RexCall invocation, String inputTerm) { + invocation.accept(new AsyncCorrelateFunctionsValidator()); + + ExprCodeGenerator exprGenerator = + new ExprCodeGenerator(ctx, false) + .bindInput( + inputType, + inputTerm, + JavaScalaConversionUtil.toScala(Optional.empty())); + + GeneratedExpression invocationExprs = exprGenerator.generateExpression(invocation); + return invocationExprs.code(); + } + + private static class AsyncCorrelateFunctionsValidator extends RexVisitorImpl { + public AsyncCorrelateFunctionsValidator() { + super(true); + } + + @Override + public Void visitCall(RexCall call) { + super.visitCall(call); + + if (call.getOperator() instanceof BridgingSqlFunction + && ((BridgingSqlFunction) call.getOperator()).getDefinition().getKind() + != FunctionKind.ASYNC_TABLE) { + throw new CodeGenException( + "Invalid use of function " + + call.getOperator() + + "." + + "Code generation should only be done with async table calls"); + } + return null; + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java index fb4833f55b36b..0eb86538c0ef4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java @@ -126,13 +126,16 @@ public static BridgingSqlFunction of( functionKind == FunctionKind.SCALAR || functionKind == FunctionKind.ASYNC_SCALAR || functionKind == FunctionKind.TABLE + || functionKind == FunctionKind.ASYNC_TABLE || functionKind == FunctionKind.PROCESS_TABLE, "Scalar or table function kind expected."); final TypeInference systemTypeInference = SystemTypeInference.of(functionKind, typeInference); - if (functionKind == FunctionKind.TABLE || functionKind == FunctionKind.PROCESS_TABLE) { + if (functionKind == FunctionKind.TABLE + || functionKind == FunctionKind.ASYNC_TABLE + || functionKind == FunctionKind.PROCESS_TABLE) { return new BridgingSqlFunction.WithTableFunction( dataTypeFactory, typeFactory, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecAsyncCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecAsyncCalc.java index 1851e2ab7869b..93d26866514e3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecAsyncCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecAsyncCalc.java @@ -152,7 +152,7 @@ private OneInputStreamOperatorFactory getAsyncFunctionOperator config, classLoader); AsyncFunctionRunner func = new AsyncFunctionRunner(generatedFunction); - AsyncUtil.Options options = AsyncUtil.getAsyncOptions(config); + AsyncUtil.Options options = AsyncUtil.getAsyncScalarOptions(config); return new AsyncWaitOperatorFactory<>( func, options.asyncTimeout, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecAsyncCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecAsyncCorrelate.java new file mode 100644 index 0000000000000..e8ed7dcbf4a19 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecAsyncCorrelate.java @@ -0,0 +1,154 @@ +/* + * 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.planner.plan.nodes.exec.common; + +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorFactory; +import org.apache.flink.streaming.api.transformations.OneInputTransformation; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.codegen.AsyncCorrelateCodeGenerator; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator; +import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.planner.plan.utils.AsyncUtil; +import org.apache.flink.table.runtime.generated.GeneratedFunction; +import org.apache.flink.table.runtime.operators.correlate.async.AsyncCorrelateRunner; +import org.apache.flink.table.runtime.operators.join.FlinkJoinType; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.calcite.rex.RexCall; + +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** Base class for exec Async Correlate. */ +public class CommonExecAsyncCorrelate extends ExecNodeBase + implements SingleTransformationTranslator { + + public static final String ASYNC_CORRELATE_TRANSFORMATION = "async-correlate"; + + public static final String FIELD_NAME_JOIN_TYPE = "joinType"; + public static final String FIELD_NAME_FUNCTION_CALL = "functionCall"; + + @JsonProperty(FIELD_NAME_JOIN_TYPE) + private final FlinkJoinType joinType; + + @JsonProperty(FIELD_NAME_FUNCTION_CALL) + private final RexCall invocation; + + public CommonExecAsyncCorrelate( + int id, + ExecNodeContext context, + ReadableConfig persistedConfig, + FlinkJoinType joinType, + RexCall invocation, + List inputProperties, + RowType outputType, + String description) { + super(id, context, persistedConfig, inputProperties, outputType, description); + checkArgument(inputProperties.size() == 1); + this.joinType = joinType; + this.invocation = invocation; + } + + @Override + protected Transformation translateToPlanInternal( + PlannerBase planner, ExecNodeConfig config) { + final ExecEdge inputEdge = getInputEdges().get(0); + final Transformation inputTransform = + (Transformation) inputEdge.translateToPlan(planner); + final OneInputTransformation transform = + createAsyncOneInputTransformation( + inputTransform, config, planner.getFlinkContext().getClassLoader()); + + return transform; + } + + private OneInputTransformation createAsyncOneInputTransformation( + Transformation inputTransform, + ExecNodeConfig config, + ClassLoader classLoader) { + final ExecEdge inputEdge = getInputEdges().get(0); + RowType inputRowType = + RowType.of(inputEdge.getOutputType().getChildren().toArray(new LogicalType[0])); + + InternalTypeInfo asyncOperatorResultTypeInfo = + InternalTypeInfo.of(getOutputType()); + OneInputStreamOperatorFactory factory = + getAsyncFunctionOperator(config, classLoader, inputRowType); + return ExecNodeUtil.createOneInputTransformation( + inputTransform, + createTransformationMeta(ASYNC_CORRELATE_TRANSFORMATION, config), + factory, + asyncOperatorResultTypeInfo, + inputTransform.getParallelism(), + false); + } + + private OneInputStreamOperatorFactory getAsyncFunctionOperator( + ExecNodeConfig config, ClassLoader classLoader, RowType inputRowType) { + + RowType resultTypeInfo = (RowType) FlinkTypeFactory.toLogicalType(invocation.getType()); + + GeneratedFunction> generatedFunction = + AsyncCorrelateCodeGenerator.generateFunction( + "AsyncTableFunction", + inputRowType, + resultTypeInfo, + invocation, + config, + classLoader); + DataStructureConverter fetcherConverter = + cast( + DataStructureConverters.getConverter( + TypeConversions.fromLogicalToDataType( + FlinkTypeFactory.toLogicalType(invocation.getType())))); + AsyncCorrelateRunner func = new AsyncCorrelateRunner(generatedFunction, fetcherConverter); + AsyncUtil.Options options = AsyncUtil.getAsyncTableOptions(config); + return new AsyncWaitOperatorFactory<>( + func, + options.asyncTimeout, + options.asyncBufferCapacity, + options.asyncOutputMode, + options.asyncRetryStrategy); + } + + @SuppressWarnings("unchecked") + private DataStructureConverter cast( + DataStructureConverter converter) { + return (DataStructureConverter) (Object) converter; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAsyncCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAsyncCorrelate.java new file mode 100644 index 0000000000000..c3b4e448b666b --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAsyncCorrelate.java @@ -0,0 +1,93 @@ +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +/* + * 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. + */ + +import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecAsyncCorrelate; +import org.apache.flink.table.runtime.operators.join.FlinkJoinType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexNode; + +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecAsyncCorrelate.ASYNC_CORRELATE_TRANSFORMATION; + +/** + * Stream {@link ExecNode} which matches along with join a Java/Scala user defined table function. + */ +@ExecNodeMetadata( + name = "stream-exec-async-correlate", + version = 1, + producedTransformations = ASYNC_CORRELATE_TRANSFORMATION, + minPlanVersion = FlinkVersion.V2_1, + minStateVersion = FlinkVersion.V2_1) +public class StreamExecAsyncCorrelate extends CommonExecAsyncCorrelate + implements StreamExecNode { + + public StreamExecAsyncCorrelate( + ReadableConfig tableConfig, + FlinkJoinType joinType, + RexCall invocation, + InputProperty inputProperty, + RowType outputType, + String description) { + this( + ExecNodeContext.newNodeId(), + ExecNodeContext.newContext(StreamExecAsyncCorrelate.class), + ExecNodeContext.newPersistedConfig(StreamExecAsyncCorrelate.class, tableConfig), + joinType, + invocation, + Collections.singletonList(inputProperty), + outputType, + description); + } + + @JsonCreator + public StreamExecAsyncCorrelate( + @JsonProperty(FIELD_NAME_ID) int id, + @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, + @JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType, + @JsonProperty(FIELD_NAME_FUNCTION_CALL) RexNode invocation, + @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, + @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, + @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { + super( + id, + context, + persistedConfig, + joinType, + (RexCall) invocation, + inputProperties, + outputType, + description); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.java index 19287e0828714..d478b3afcf072 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.java @@ -101,6 +101,7 @@ public boolean matches(RelOptRuleCall call) { } final boolean isTableFunction = functionDefinition.getKind() == FunctionKind.TABLE + || functionDefinition.getKind() == FunctionKind.ASYNC_TABLE || functionDefinition.getKind() == FunctionKind.PROCESS_TABLE; return isTableFunction && !(functionDefinition instanceof TemporalTableFunction); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalAsyncCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalAsyncCorrelate.java new file mode 100644 index 0000000000000..20059c74ae8f5 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalAsyncCorrelate.java @@ -0,0 +1,95 @@ +/* + * 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.planner.plan.nodes.physical.stream; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCorrelate; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan; +import org.apache.flink.table.planner.plan.utils.JoinTypeUtil; +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexNode; + +import java.util.List; +import java.util.Optional; + +import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig; + +/** Stream physical RelNode for {@link org.apache.flink.table.functions.AsyncScalarFunction}. */ +public class StreamPhysicalAsyncCorrelate extends StreamPhysicalCorrelateBase { + private final RelOptCluster cluster; + private final FlinkLogicalTableFunctionScan scan; + private final Optional> projections; + private final Optional condition; + private final JoinRelType joinType; + + public StreamPhysicalAsyncCorrelate( + RelOptCluster cluster, + RelTraitSet traitSet, + RelNode inputRel, + FlinkLogicalTableFunctionScan scan, + Optional> projections, + Optional condition, + RelDataType outputRowType, + JoinRelType joinType) { + super( + cluster, + traitSet, + inputRel, + scan, + JavaScalaConversionUtil.toScala(condition), + outputRowType, + joinType); + this.cluster = cluster; + this.scan = scan; + this.projections = projections; + this.condition = condition; + this.joinType = joinType; + } + + @Override + public RelNode copy(RelTraitSet traitSet, RelNode newChild, RelDataType outputType) { + return new StreamPhysicalAsyncCorrelate( + cluster, traitSet, newChild, scan, projections, condition, outputType, joinType); + } + + @Override + public ExecNode translateToExecNode() { + if (projections.isPresent() || condition.isPresent()) { + throw new TableException( + "Currently Async correlate does not support projections or conditions."); + } + return new StreamExecAsyncCorrelate( + unwrapTableConfig(this), + JoinTypeUtil.getFlinkJoinType(joinType), + (RexCall) scan.getCall(), + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType()), + getRelDetailedDescription()); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRule.java index cc7c72a909eb7..e4a6fac5a5117 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRule.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.rules.logical; +import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc; import org.apache.flink.table.planner.plan.utils.AsyncUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; @@ -41,7 +42,8 @@ */ public class AsyncCalcSplitRule { - private static final RemoteCalcCallFinder ASYNC_CALL_FINDER = new AsyncRemoteCalcCallFinder(); + private static final RemoteCalcCallFinder ASYNC_CALL_FINDER = + new AsyncRemoteCalcCallFinder(FunctionKind.ASYNC_SCALAR); public static final RelOptRule SPLIT_CONDITION = new RemoteCalcSplitConditionRule(ASYNC_CALL_FINDER); public static final RelOptRule SPLIT_PROJECT = @@ -68,27 +70,35 @@ public class AsyncCalcSplitRule { /** * An Async implementation of {@link RemoteCalcCallFinder} which finds uses of {@link - * org.apache.flink.table.functions.AsyncScalarFunction}. + * org.apache.flink.table.functions.AsyncScalarFunction} and {@link + * org.apache.flink.table.functions.AsyncTableFunction}. */ public static class AsyncRemoteCalcCallFinder implements RemoteCalcCallFinder { + + private final FunctionKind functionKind; + + public AsyncRemoteCalcCallFinder(FunctionKind functionKind) { + this.functionKind = functionKind; + } + @Override public boolean containsRemoteCall(RexNode node) { - return AsyncUtil.containsAsyncCall(node); + return AsyncUtil.containsAsyncCall(node, functionKind); } @Override public boolean containsNonRemoteCall(RexNode node) { - return AsyncUtil.containsNonAsyncCall(node); + return AsyncUtil.containsNonAsyncCall(node, functionKind); } @Override public boolean isRemoteCall(RexNode node) { - return AsyncUtil.isAsyncCall(node); + return AsyncUtil.isAsyncCall(node, functionKind); } @Override public boolean isNonRemoteCall(RexNode node) { - return AsyncUtil.isNonAsyncCall(node); + return AsyncUtil.isNonAsyncCall(node, functionKind); } @Override @@ -98,7 +108,9 @@ public String getName() { @Override public boolean equals(Object obj) { - return obj != null && this.getClass() == obj.getClass(); + return obj != null + && this.getClass() == obj.getClass() + && functionKind == ((AsyncRemoteCalcCallFinder) obj).functionKind; } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRule.java index 0094094c0f17f..4008bbb597b25 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRule.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.rules.logical; +import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc; import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCorrelate; import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan; @@ -33,8 +34,13 @@ */ public class AsyncCorrelateSplitRule { - private static final RemoteCalcCallFinder ASYNC_CALL_FINDER = new AsyncRemoteCalcCallFinder(); + private static final RemoteCalcCallFinder ASYNC_SCALAR_CALL_FINDER = + new AsyncRemoteCalcCallFinder(FunctionKind.ASYNC_SCALAR); + private static final RemoteCalcCallFinder ASYNC_TABLE_CALL_FINDER = + new AsyncRemoteCalcCallFinder(FunctionKind.ASYNC_TABLE); - public static final RelOptRule INSTANCE = - RemoteCorrelateSplitRule.Config.createDefault(ASYNC_CALL_FINDER).toRule(); + public static final RelOptRule CORRELATE_SPLIT_ASYNC_SCALAR = + RemoteCorrelateSplitRule.Config.createDefault(ASYNC_SCALAR_CALL_FINDER).toRule(); + public static final RelOptRule CORRELATE_SPLIT_ASYNC_TABLE = + RemoteCorrelateSplitRule.Config.createDefault(ASYNC_TABLE_CALL_FINDER).toRule(); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RemoteCorrelateSplitRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RemoteCorrelateSplitRule.java index d12d72f775539..fc63aa286f495 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RemoteCorrelateSplitRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RemoteCorrelateSplitRule.java @@ -137,7 +137,7 @@ public RexNode visitCall(RexCall call) { call.getOperands().stream() .map(x -> x.accept(this)) .collect(Collectors.toList()); - return rexBuilder.makeCall(call.getOperator(), newProjects); + return rexBuilder.makeCall(call.getType(), call.getOperator(), newProjects); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalAsyncCorrelateRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalAsyncCorrelateRule.java new file mode 100644 index 0000000000000..38ecc96188dab --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalAsyncCorrelateRule.java @@ -0,0 +1,150 @@ +/* + * 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.planner.plan.rules.physical.stream; + +import org.apache.flink.table.functions.FunctionKind; +import org.apache.flink.table.planner.plan.nodes.FlinkConventions; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCorrelate; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalAsyncCorrelate; +import org.apache.flink.table.planner.plan.utils.AsyncUtil; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.plan.volcano.RelSubset; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rex.RexNode; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * A physical rule for identifying logical correlates containing {@link + * org.apache.flink.table.functions.AsyncTableFunction} calls and converting them to physical {@link + * StreamPhysicalAsyncCorrelate} RelNodes. + */ +public class StreamPhysicalAsyncCorrelateRule extends ConverterRule { + + public static final RelOptRule INSTANCE = + new StreamPhysicalAsyncCorrelateRule( + Config.INSTANCE.withConversion( + FlinkLogicalCorrelate.class, + FlinkConventions.LOGICAL(), + FlinkConventions.STREAM_PHYSICAL(), + "StreamPhysicalAsyncCorrelateRule")); + + protected StreamPhysicalAsyncCorrelateRule(Config config) { + super(config); + } + + // find only calc and table function + private boolean findAsyncTableFunction(FlinkLogicalCalc calc) { + RelNode child = ((RelSubset) calc.getInput()).getOriginal(); + if (child instanceof FlinkLogicalTableFunctionScan) { + FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) child; + return AsyncUtil.isAsyncCall(scan.getCall(), FunctionKind.ASYNC_TABLE); + } else if (child instanceof FlinkLogicalCalc) { + FlinkLogicalCalc childCalc = (FlinkLogicalCalc) child; + return findAsyncTableFunction(childCalc); + } + return false; + } + + @Override + public boolean matches(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + RelNode right = ((RelSubset) correlate.getRight()).getOriginal(); + if (right instanceof FlinkLogicalTableFunctionScan) { + // right node is a table function + FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) right; + return AsyncUtil.isAsyncCall(scan.getCall(), FunctionKind.ASYNC_TABLE); + } else if (right instanceof FlinkLogicalCalc) { + // a filter is pushed above the table function + return findAsyncTableFunction((FlinkLogicalCalc) right); + } + return false; + } + + @Override + public RelNode convert(RelNode rel) { + FlinkLogicalCorrelate correlate = (FlinkLogicalCorrelate) rel; + RelTraitSet traitSet = rel.getTraitSet().replace(FlinkConventions.STREAM_PHYSICAL()); + RelNode convInput = + RelOptRule.convert(correlate.getInput(0), FlinkConventions.STREAM_PHYSICAL()); + RelNode right = correlate.getInput(1); + return convertToCorrelate( + right, correlate, traitSet, convInput, Optional.empty(), Optional.empty()); + } + + public RelNode convertToCorrelate( + RelNode relNode, + FlinkLogicalCorrelate correlate, + RelTraitSet traitSet, + RelNode convInput, + Optional> projections, + Optional condition) { + if (relNode instanceof RelSubset) { + RelSubset rel = (RelSubset) relNode; + return convertToCorrelate( + rel.getRelList().get(0), + correlate, + traitSet, + convInput, + projections, + condition); + } else if (relNode instanceof FlinkLogicalCalc) { + FlinkLogicalCalc calc = (FlinkLogicalCalc) relNode; + RelNode tableScan = StreamPhysicalCorrelateRule.getTableScan(calc); + FlinkLogicalCalc newCalc = StreamPhysicalCorrelateRule.getMergedCalc(calc); + // The projections are not handled here or in the base version, so currently we match + // that functionality. + return convertToCorrelate( + tableScan, + correlate, + traitSet, + convInput, + Optional.ofNullable( + newCalc.getProgram().getProjectList() == null + ? null + : newCalc.getProgram().getProjectList().stream() + .map(newCalc.getProgram()::expandLocalRef) + .collect(Collectors.toList())), + Optional.ofNullable( + newCalc.getProgram().getCondition() == null + ? null + : newCalc.getProgram() + .expandLocalRef(newCalc.getProgram().getCondition()))); + } else { + FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) relNode; + return new StreamPhysicalAsyncCorrelate( + correlate.getCluster(), + traitSet, + convInput, + scan, + projections, + condition, + correlate.getRowType(), + correlate.getJoinType()); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalConstantTableFunctionScanRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalConstantTableFunctionScanRule.java index 10b06744415a4..761aff5d80d53 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalConstantTableFunctionScanRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalConstantTableFunctionScanRule.java @@ -23,6 +23,7 @@ import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.planner.plan.nodes.FlinkConventions; import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalAsyncCorrelate; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelate; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalProcessTableFunction; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues; @@ -38,6 +39,8 @@ import org.apache.calcite.rex.RexUtil; import org.immutables.value.Value; +import java.util.Optional; + import scala.Option; /** @@ -110,6 +113,17 @@ public void onMatch(RelOptRuleCall call) { Option.empty(), scan.getRowType(), JoinRelType.INNER); + } else if (function.getKind() == FunctionKind.ASYNC_TABLE) { + replacement = + new StreamPhysicalAsyncCorrelate( + cluster, + traitSet, + values, + scan, + Optional.empty(), + Optional.empty(), + scan.getRowType(), + JoinRelType.INNER); } else if (function.getKind() == FunctionKind.PROCESS_TABLE) { replacement = new StreamPhysicalProcessTableFunction( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/AsyncUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/AsyncUtil.java index 61c368ff498e4..92add5414afb6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/AsyncUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/AsyncUtil.java @@ -27,6 +27,7 @@ import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig; import org.apache.flink.table.planner.utils.ShortcutUtils; +import org.apache.flink.util.Preconditions; import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexNode; @@ -47,7 +48,14 @@ public class AsyncUtil { * @return true if it contains an async function call in the specified node. */ public static boolean containsAsyncCall(RexNode node) { - return node.accept(new FunctionFinder(true, true)); + return node.accept(new FunctionFinder(true, true, null)); + } + + public static boolean containsAsyncCall(RexNode node, FunctionKind functionKind) { + Preconditions.checkArgument( + functionKind == FunctionKind.ASYNC_SCALAR + || functionKind == FunctionKind.ASYNC_TABLE); + return node.accept(new FunctionFinder(true, true, functionKind)); } /** @@ -57,7 +65,14 @@ public static boolean containsAsyncCall(RexNode node) { * @return true if it contains a non-async function call in the specified node. */ public static boolean containsNonAsyncCall(RexNode node) { - return node.accept(new FunctionFinder(false, true)); + return node.accept(new FunctionFinder(false, true, null)); + } + + public static boolean containsNonAsyncCall(RexNode node, FunctionKind functionKind) { + Preconditions.checkArgument( + functionKind == FunctionKind.ASYNC_SCALAR + || functionKind == FunctionKind.ASYNC_TABLE); + return node.accept(new FunctionFinder(false, true, functionKind)); } /** @@ -67,7 +82,14 @@ public static boolean containsNonAsyncCall(RexNode node) { * @return true if the specified node is an async function call. */ public static boolean isAsyncCall(RexNode node) { - return node.accept(new FunctionFinder(true, false)); + return node.accept(new FunctionFinder(true, false, null)); + } + + public static boolean isAsyncCall(RexNode node, FunctionKind functionKind) { + Preconditions.checkArgument( + functionKind == FunctionKind.ASYNC_SCALAR + || functionKind == FunctionKind.ASYNC_TABLE); + return node.accept(new FunctionFinder(true, false, functionKind)); } /** @@ -77,7 +99,14 @@ public static boolean isAsyncCall(RexNode node) { * @return true if the specified node is a non-async function call. */ public static boolean isNonAsyncCall(RexNode node) { - return node.accept(new FunctionFinder(false, false)); + return node.accept(new FunctionFinder(false, false, null)); + } + + public static boolean isNonAsyncCall(RexNode node, FunctionKind functionKind) { + Preconditions.checkArgument( + functionKind == FunctionKind.ASYNC_SCALAR + || functionKind == FunctionKind.ASYNC_TABLE); + return node.accept(new FunctionFinder(false, false, functionKind)); } /** @@ -86,7 +115,7 @@ public static boolean isNonAsyncCall(RexNode node) { * @param config The config from which to fetch the options * @return Extracted options */ - public static AsyncUtil.Options getAsyncOptions(ExecNodeConfig config) { + public static AsyncUtil.Options getAsyncScalarOptions(ExecNodeConfig config) { return new AsyncUtil.Options( config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_SCALAR_BUFFER_CAPACITY), config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_SCALAR_TIMEOUT).toMillis(), @@ -97,6 +126,23 @@ public static AsyncUtil.Options getAsyncOptions(ExecNodeConfig config) { config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_SCALAR_MAX_ATTEMPTS))); } + /** + * Gets the options required to run the operator. + * + * @param config The config from which to fetch the options + * @return Extracted options + */ + public static AsyncUtil.Options getAsyncTableOptions(ExecNodeConfig config) { + return new AsyncUtil.Options( + config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_BUFFER_CAPACITY), + config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_TIMEOUT).toMillis(), + AsyncDataStream.OutputMode.ORDERED, + getResultRetryStrategy( + config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_RETRY_STRATEGY), + config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_RETRY_DELAY), + config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_MAX_ATTEMPTS))); + } + /** Options for configuring async behavior. */ public static class Options { @@ -161,10 +207,12 @@ private static class FunctionFinder extends RexDefaultVisitor { private final boolean findAsyncCall; private final boolean recursive; + private final FunctionKind functionKind; - public FunctionFinder(boolean findAsyncCall, boolean recursive) { + public FunctionFinder(boolean findAsyncCall, boolean recursive, FunctionKind functionKind) { this.findAsyncCall = findAsyncCall; this.recursive = recursive; + this.functionKind = functionKind; } @Override @@ -174,7 +222,11 @@ public Boolean visitNode(RexNode rexNode) { private boolean isImmediateAsyncCall(RexCall call) { FunctionDefinition definition = ShortcutUtils.unwrapFunctionDefinition(call); - return definition != null && definition.getKind() == FunctionKind.ASYNC_SCALAR; + return definition != null + && ((functionKind != null && definition.getKind() == functionKind) + || (functionKind == null + && (definition.getKind() == FunctionKind.ASYNC_SCALAR + || definition.getKind() == FunctionKind.ASYNC_TABLE))); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java index 484d0883be1b6..b3571d3cff23e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java @@ -49,6 +49,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecValues; import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecWindowTableFunction; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCalc; +import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCorrelate; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate; @@ -166,6 +167,7 @@ private ExecNodeMetadataUtil() { add(StreamExecPythonCalc.class); add(StreamExecAsyncCalc.class); add(StreamExecProcessTableFunction.class); + add(StreamExecAsyncCorrelate.class); add(StreamExecPythonCorrelate.class); add(StreamExecPythonGroupAggregate.class); add(StreamExecPythonGroupWindowAggregate.class); diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingFunctionGenUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingFunctionGenUtil.scala index 465e5220bf248..d1c6e50fc5fb2 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingFunctionGenUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingFunctionGenUtil.scala @@ -36,7 +36,7 @@ import org.apache.flink.table.planner.utils.JavaScalaConversionUtil.toScala import org.apache.flink.table.runtime.collector.WrappingCollector import org.apache.flink.table.runtime.functions.DefaultExpressionEvaluator import org.apache.flink.table.runtime.generated.GeneratedFunction -import org.apache.flink.table.runtime.operators.join.lookup.DelegatingResultFuture +import org.apache.flink.table.runtime.operators.correlate.async.DelegatingAsyncTableResultFuture import org.apache.flink.table.types.DataType import org.apache.flink.table.types.extraction.ExtractionUtils.primitiveToWrapper import org.apache.flink.table.types.inference.{CallContext, TypeInference, TypeInferenceUtil} @@ -44,10 +44,11 @@ import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot, RowTy import org.apache.flink.table.types.logical.RowType.RowField import org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isCompositeType +import org.apache.flink.table.types.utils.DataTypeUtils import org.apache.flink.table.types.utils.DataTypeUtils.{isInternal, validateInputDataType, validateOutputDataType} import org.apache.flink.util.Preconditions -import AsyncCodeGenerator.DEFAULT_DELEGATING_FUTURE_TERM +import AsyncCodeGenerator.{generateFunction, DEFAULT_DELEGATING_FUTURE_TERM} import java.util.concurrent.CompletableFuture @@ -152,7 +153,7 @@ object BridgingFunctionGenUtil { contextTerm ) } else if (udf.getKind == FunctionKind.ASYNC_TABLE) { - generateAsyncTableFunctionCall(functionTerm, externalOperands, returnType) + generateAsyncTableFunctionCall(functionTerm, externalOperands, returnType, outputDataType) } else if (udf.getKind == FunctionKind.ASYNC_SCALAR) { generateAsyncScalarFunctionCall( ctx, @@ -205,25 +206,36 @@ object BridgingFunctionGenUtil { private def generateAsyncTableFunctionCall( functionTerm: String, externalOperands: Seq[GeneratedExpression], - outputType: LogicalType): GeneratedExpression = { + returnType: LogicalType, + outputDataType: DataType): GeneratedExpression = { + + val DELEGATE_ASYNC_TABLE = className[DelegatingAsyncTableResultFuture] + val outputType = outputDataType.getLogicalType - val DELEGATE = className[DelegatingResultFuture[_]] + // If we need to wrap data in a row, it's done in the delegating class. + val needsWrapping = !isCompositeType(outputType) + val isInternal = DataTypeUtils.isInternal(outputDataType); + val arguments = Seq( + s""" + |delegates.getCompletableFuture() + |""".stripMargin + ) ++ externalOperands.map(_.resultTerm) + val anyNull = externalOperands.map(_.nullTerm) ++ Seq("false") val functionCallCode = s""" |${externalOperands.map(_.code).mkString("\n")} - |if (${externalOperands.map(_.nullTerm).mkString(" || ")}) { + |if (${anyNull.mkString(" || ")}) { | $DEFAULT_COLLECTOR_TERM.complete(java.util.Collections.emptyList()); |} else { - | $DELEGATE delegates = new $DELEGATE($DEFAULT_COLLECTOR_TERM); - | $functionTerm.eval( - | delegates.getCompletableFuture(), - | ${externalOperands.map(_.resultTerm).mkString(", ")}); + | $DELEGATE_ASYNC_TABLE delegates = new $DELEGATE_ASYNC_TABLE($DEFAULT_COLLECTOR_TERM, + | $needsWrapping, $isInternal); + | $functionTerm.eval(${arguments.mkString(", ")}); |} |""".stripMargin // has no result - GeneratedExpression(NO_CODE, NEVER_NULL, functionCallCode, outputType) + GeneratedExpression(NO_CODE, NEVER_NULL, functionCallCode, returnType) } private def generateAsyncScalarFunctionCall( @@ -377,7 +389,7 @@ object BridgingFunctionGenUtil { udf: UserDefinedFunction): Unit = { val enrichedType = enrichedDataType.getLogicalType if ( - (udf.getKind == FunctionKind.TABLE || udf.getKind == FunctionKind.PROCESS_TABLE) && !isCompositeType( + (udf.getKind == FunctionKind.TABLE || udf.getKind == FunctionKind.ASYNC_TABLE || udf.getKind == FunctionKind.PROCESS_TABLE) && !isCompositeType( enrichedType) ) { // logically table functions wrap atomic types into ROW, however, the physical function might @@ -389,10 +401,6 @@ object BridgingFunctionGenUtil { ) val atomicOutputType = returnType.asInstanceOf[RowType].getChildren.get(0) verifyOutputType(atomicOutputType, enrichedDataType) - } else if (udf.getKind == FunctionKind.ASYNC_TABLE && !isCompositeType(enrichedType)) { - throw new CodeGenException( - "Async table functions must not emit an atomic type. " + - "Only a composite type such as the row type are supported.") } else if ( udf.getKind == FunctionKind.TABLE || udf.getKind == FunctionKind.PROCESS_TABLE || udf.getKind == FunctionKind.ASYNC_TABLE ) { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index 74219150ed0fa..3e3dd3c7d53ed 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.rules import org.apache.flink.table.planner.plan.nodes.logical._ import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalProcessTableFunctionRule import org.apache.flink.table.planner.plan.rules.logical._ +import org.apache.flink.table.planner.plan.rules.logical.{AsyncCorrelateSplitRule, _} import org.apache.flink.table.planner.plan.rules.physical.FlinkExpandConversionRule import org.apache.flink.table.planner.plan.rules.physical.stream._ @@ -398,6 +399,10 @@ object FlinkStreamRuleSets { PythonMapMergeRule.INSTANCE, // Similar to the python rules above, the goal is to limit complexity of calcs which // have async calls so that the implementation can be simplified to handle a single async call. + // Split async scalar calls from other types of calls in correlates + AsyncCorrelateSplitRule.CORRELATE_SPLIT_ASYNC_SCALAR, + // Split async table calls from other types of calls in correlates + AsyncCorrelateSplitRule.CORRELATE_SPLIT_ASYNC_TABLE, // Avoids accessing a field from an asynchronous result (condition). AsyncCalcSplitRule.SPLIT_CONDITION_REX_FIELD, // Avoids accessing a field from an asynchronous result (projection). @@ -417,9 +422,7 @@ object FlinkStreamRuleSets { // Avoid async calls which call async calls. AsyncCalcSplitRule.NESTED_SPLIT, // Avoid having async calls in multiple projections in a single calc. - AsyncCalcSplitRule.ONE_PER_CALC_SPLIT, - // Split async calls from correlates - AsyncCorrelateSplitRule.INSTANCE + AsyncCalcSplitRule.ONE_PER_CALC_SPLIT ) /** RuleSet to do physical optimize for stream */ @@ -483,6 +486,7 @@ object FlinkStreamRuleSets { StreamPhysicalConstantTableFunctionScanRule.INSTANCE, StreamPhysicalCorrelateRule.INSTANCE, StreamPhysicalPythonCorrelateRule.INSTANCE, + StreamPhysicalAsyncCorrelateRule.INSTANCE, // sink StreamPhysicalSinkRule.INSTANCE, StreamPhysicalLegacySinkRule.INSTANCE diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalCorrelateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalCorrelateRule.scala index 83cf99c31775a..f08696e5e865b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalCorrelateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalCorrelateRule.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalCorrelate, FlinkLogicalTableFunctionScan} import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelate import org.apache.flink.table.planner.plan.rules.physical.stream.StreamPhysicalCorrelateRule.{getMergedCalc, getTableScan} -import org.apache.flink.table.planner.plan.utils.PythonUtil +import org.apache.flink.table.planner.plan.utils.{AsyncUtil, PythonUtil} import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} import org.apache.calcite.plan.hep.HepRelVertex @@ -44,7 +44,8 @@ class StreamPhysicalCorrelateRule(config: Config) extends ConverterRule(config) def findTableFunction(calc: FlinkLogicalCalc): Boolean = { val child = calc.getInput.asInstanceOf[RelSubset].getOriginal child match { - case scan: FlinkLogicalTableFunctionScan => PythonUtil.isNonPythonCall(scan.getCall) + case scan: FlinkLogicalTableFunctionScan => + PythonUtil.isNonPythonCall(scan.getCall) && AsyncUtil.isNonAsyncCall(scan.getCall) case calc: FlinkLogicalCalc => findTableFunction(calc) case _ => false } @@ -52,7 +53,8 @@ class StreamPhysicalCorrelateRule(config: Config) extends ConverterRule(config) right match { // right node is a table function - case scan: FlinkLogicalTableFunctionScan => PythonUtil.isNonPythonCall(scan.getCall) + case scan: FlinkLogicalTableFunctionScan => + PythonUtil.isNonPythonCall(scan.getCall) && AsyncUtil.isNonAsyncCall(scan.getCall) // a filter is pushed above the table function case calc: FlinkLogicalCalc => findTableFunction(calc) case _ => false @@ -79,7 +81,9 @@ class StreamPhysicalCorrelateRule(config: Config) extends ConverterRule(config) val newCalc = getMergedCalc(calc) convertToCorrelate( tableScan, - Some(newCalc.getProgram.expandLocalRef(newCalc.getProgram.getCondition))) + if (newCalc.getProgram.getCondition == null) None + else Some(newCalc.getProgram.expandLocalRef(newCalc.getProgram.getCondition)) + ) case scan: FlinkLogicalTableFunctionScan => new StreamPhysicalCorrelate( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/codegen/AsyncCorrelateCodeGeneratorTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/codegen/AsyncCorrelateCodeGeneratorTest.java new file mode 100644 index 0000000000000..f24ebcd899287 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/codegen/AsyncCorrelateCodeGeneratorTest.java @@ -0,0 +1,245 @@ +/* + * 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.planner.codegen; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.streaming.api.functions.async.CollectionSupplier; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.FunctionHint; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.functions.AsyncTableFunction; +import org.apache.flink.table.planner.calcite.SqlToRexConverter; +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.PlannerMocks; +import org.apache.flink.table.planner.utils.ShortcutUtils; +import org.apache.flink.table.runtime.generated.GeneratedFunction; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Row; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexCall; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link AsyncCorrelateCodeGenerator}. */ +public class AsyncCorrelateCodeGeneratorTest { + + private static final RowType INPUT_TYPE = + RowType.of(new IntType(), new BigIntType(), new VarCharType()); + + private PlannerMocks plannerMocks; + private SqlToRexConverter converter; + + private RelDataType tableRowType; + + @BeforeEach + public void before() { + plannerMocks = PlannerMocks.create(); + tableRowType = + plannerMocks + .getPlannerContext() + .getTypeFactory() + .buildRelNodeRowType( + JavaScalaConversionUtil.toScala(Arrays.asList("f1", "f2", "f3")), + JavaScalaConversionUtil.toScala( + Arrays.asList( + new IntType(), + new BigIntType(), + new VarCharType()))); + ShortcutUtils.unwrapContext(plannerMocks.getPlanner().createToRelContext().getCluster()); + plannerMocks + .getFunctionCatalog() + .registerTemporarySystemFunction("myfunc", new AsyncFunc(), false); + plannerMocks + .getFunctionCatalog() + .registerTemporarySystemFunction("myfunc2", new AsyncRowFunc(), false); + plannerMocks + .getFunctionCatalog() + .registerTemporarySystemFunction("myfunc_error", new AsyncFuncError(), false); + + converter = + ShortcutUtils.unwrapContext( + plannerMocks.getPlanner().createToRelContext().getCluster()) + .getRexFactory() + .createSqlToRexConverter( + tableRowType, + plannerMocks + .getPlannerContext() + .getTypeFactory() + .createFieldTypeFromLogicalType( + RowType.of(VarCharType.STRING_TYPE))); + } + + @Test + public void testStringReturnType() throws Exception { + List objects = + execute( + "myFunc(f1, f2, f3)", + RowType.of(VarCharType.STRING_TYPE), + GenericRowData.of(2, 3L, StringData.fromString("foo"))); + assertThat(objects) + .containsExactly(Row.of("complete1 foo 4 6"), Row.of("complete2 foo 4 6")); + } + + @Test + public void testRowReturnType() throws Exception { + RowType type = RowType.of(new IntType(), new BigIntType(), VarCharType.STRING_TYPE); + List objects = + execute( + "myFunc2(f1, f2, f3)", + type, + GenericRowData.of(2, 3L, StringData.fromString("foo"))); + assertThat(objects) + .containsExactly(Row.of(2, 30L, "complete1 foo"), Row.of(2, 60L, "complete2 foo")); + + objects = + execute( + "myFunc2(f1, f2, f3)", + type, + GenericRowData.of(0, 3L, StringData.fromString("foo"))); + assertThat(objects).containsExactly(); + + objects = + execute( + "myFunc2(f1, f2, f3)", + type, + GenericRowData.of(1, 3L, StringData.fromString("foo"))); + assertThat(objects).containsExactly(Row.of(1, 30L, "complete foo")); + } + + @Test + public void testError() throws Exception { + CompletableFuture> future = + executeFuture( + "myFunc_error(f1, f2, f3)", + RowType.of(VarCharType.STRING_TYPE), + GenericRowData.of(2, 3L, StringData.fromString("foo"))); + assertThat(future).isCompletedExceptionally(); + assertThatThrownBy(future::get).cause().hasMessage("Error!"); + } + + private List execute(String sqlExpression, RowType resultType, RowData input) + throws Exception { + Collection result = executeFuture(sqlExpression, resultType, input).get(); + return new ArrayList<>(result); + } + + private CompletableFuture> executeFuture( + String sqlExpression, RowType resultType, RowData input) throws Exception { + RelDataType type = + plannerMocks.getPlannerContext().getTypeFactory().buildRelNodeRowType(resultType); + + RexCall node = (RexCall) converter.convertToRexNode(sqlExpression); + node = node.clone(type, node.getOperands()); + + GeneratedFunction> function = + AsyncCorrelateCodeGenerator.generateFunction( + "name", + INPUT_TYPE, + resultType, + node, + new Configuration(), + Thread.currentThread().getContextClassLoader()); + AsyncFunction asyncFunction = + function.newInstance(Thread.currentThread().getContextClassLoader()); + TestResultFuture resultFuture = new TestResultFuture(); + asyncFunction.asyncInvoke(input, resultFuture); + return resultFuture.getResult(); + } + + /** Test function. */ + public static final class AsyncFunc extends AsyncTableFunction { + public void eval(CompletableFuture> f, Integer i, Long l, String s) { + f.complete( + Arrays.asList( + "complete1 " + s + " " + (i * i) + " " + (2 * l), + "complete2 " + s + " " + (i * i) + " " + (2 * l))); + } + } + + /** Test function. */ + @FunctionHint(output = @DataTypeHint("ROW")) + public static final class AsyncRowFunc extends AsyncTableFunction { + public void eval(CompletableFuture> f, Integer i, Long l, String s) { + if (i == 0) { + f.complete(Collections.emptyList()); + } else if (i == 1) { + f.complete(Collections.singletonList(Row.of(i, l * 10, "complete " + s))); + } else { + f.complete( + Arrays.asList( + Row.of(i, l * 10, "complete1 " + s), + Row.of(i, l * 20, "complete2 " + s))); + } + } + } + + /** Test function. */ + public static final class AsyncFuncError extends AsyncTableFunction { + public void eval(CompletableFuture> f, Integer i, Long l, String s) { + f.completeExceptionally(new RuntimeException("Error!")); + } + } + + /** Test result future. */ + public static final class TestResultFuture implements ResultFuture { + + CompletableFuture> data = new CompletableFuture<>(); + + @Override + public void complete(Collection result) { + data.complete(result); + } + + @Override + public void completeExceptionally(Throwable error) { + data.completeExceptionally(error); + } + + public CompletableFuture> getResult() { + return data; + } + + @Override + public void complete(CollectionSupplier supplier) { + try { + data.complete(supplier.get()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/AsyncCorrelateRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/AsyncCorrelateRestoreTest.java new file mode 100644 index 0000000000000..2d52118b22526 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/AsyncCorrelateRestoreTest.java @@ -0,0 +1,43 @@ +/* + * 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.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.planner.plan.nodes.exec.testutils.RestoreTestBase; +import org.apache.flink.table.test.program.TableTestProgram; + +import java.util.Arrays; +import java.util.List; + +/** Restore tests for {@link StreamExecAsyncCorrelate}. */ +public class AsyncCorrelateRestoreTest extends RestoreTestBase { + + public AsyncCorrelateRestoreTest() { + super(StreamExecCorrelate.class); + } + + @Override + public List programs() { + return Arrays.asList( + AsyncCorrelateTestPrograms.CORRELATE_CATALOG_FUNC, + AsyncCorrelateTestPrograms.CORRELATE_SYSTEM_FUNC, + AsyncCorrelateTestPrograms.CORRELATE_JOIN_FILTER, + AsyncCorrelateTestPrograms.CORRELATE_LEFT_JOIN, + AsyncCorrelateTestPrograms.CORRELATE_UDF_EXCEPTION); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/AsyncCorrelateTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/AsyncCorrelateTestPrograms.java new file mode 100644 index 0000000000000..ea8ec43503f7f --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/AsyncCorrelateTestPrograms.java @@ -0,0 +1,238 @@ +/* + * 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.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.functions.AsyncTableFunction; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.AsyncStringSplit; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_BUFFER_CAPACITY; +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_MAX_ATTEMPTS; +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_RETRY_DELAY; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecAsyncCorrelate}. */ +public class AsyncCorrelateTestPrograms { + static final Row[] BEFORE_DATA = {Row.of(1L, 1, "hi#there"), Row.of(2L, 2, "hello#world")}; + + static final Row[] AFTER_DATA = { + Row.of(4L, 4, "foo#bar"), Row.of(3L, 3, "bar#fiz"), + }; + + static final String[] SOURCE_SCHEMA = {"a BIGINT", "b INT NOT NULL", "c VARCHAR"}; + + public static final TableTestProgram CORRELATE_CATALOG_FUNC = + TableTestProgram.of( + "async-correlate-catalog-func", + "validate correlate with temporary catalog function") + .setupTemporaryCatalogFunction("func1", TableFunc1.class) + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema(SOURCE_SCHEMA) + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a VARCHAR", "b VARCHAR") + .consumedBeforeRestore( + "+I[hi#there, $hi]", + "+I[hi#there, $there]", + "+I[hello#world, $hello]", + "+I[hello#world, $world]") + .consumedAfterRestore( + "+I[foo#bar, $foo]", + "+I[foo#bar, $bar]", + "+I[bar#fiz, $bar]", + "+I[bar#fiz, $fiz]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT c, s FROM source_t, LATERAL TABLE(func1(c, '$')) AS T(s)") + .build(); + + public static final TableTestProgram CORRELATE_SYSTEM_FUNC = + TableTestProgram.of( + "async-correlate-system-func", + "validate correlate with temporary system function") + .setupTemporarySystemFunction("STRING_SPLIT", AsyncStringSplit.class) + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema(SOURCE_SCHEMA) + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a VARCHAR", "b VARCHAR") + .consumedBeforeRestore( + "+I[hi#there, hi]", + "+I[hi#there, there]", + "+I[hello#world, hello]", + "+I[hello#world, world]") + .consumedAfterRestore( + "+I[foo#bar, foo]", + "+I[foo#bar, bar]", + "+I[bar#fiz, bar]", + "+I[bar#fiz, fiz]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT c, s FROM source_t, LATERAL TABLE(STRING_SPLIT(c, '#')) AS T(s)") + .build(); + + public static final TableTestProgram CORRELATE_JOIN_FILTER = + TableTestProgram.of( + "async-correlate-join-filter", + "validate correlate with join and filter") + .setupTemporaryCatalogFunction("func1", TableFunc1.class) + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema(SOURCE_SCHEMA) + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a VARCHAR", "b VARCHAR") + .consumedBeforeRestore( + "+I[hello#world, hello]", "+I[hello#world, world]") + .consumedAfterRestore("+I[bar#fiz, bar]", "+I[bar#fiz, fiz]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT * FROM (SELECT c, s FROM source_t, LATERAL TABLE(func1(c)) AS T(s)) AS T2 WHERE c LIKE '%hello%' OR c LIKE '%fiz%'") + .build(); + + public static final TableTestProgram CORRELATE_LEFT_JOIN = + TableTestProgram.of("async-correlate-left-join", "validate correlate with left join") + .setupTemporaryCatalogFunction("func1", TableFunc1.class) + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema(SOURCE_SCHEMA) + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a VARCHAR", "b VARCHAR") + .consumedBeforeRestore( + "+I[hi#there, hi]", + "+I[hi#there, there]", + "+I[hello#world, hello]", + "+I[hello#world, world]") + .consumedAfterRestore( + "+I[foo#bar, foo]", + "+I[foo#bar, bar]", + "+I[bar#fiz, bar]", + "+I[bar#fiz, fiz]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT c, s FROM source_t LEFT JOIN LATERAL TABLE(func1(c)) AS T(s) ON TRUE") + .build(); + + public static final TableTestProgram CORRELATE_UDF_EXCEPTION = + TableTestProgram.of( + "async-correlate-exception", + "validates async calc node that fails some number of times and then recovers after restore") + .setupConfig(TABLE_EXEC_ASYNC_TABLE_RETRY_DELAY, Duration.ofMillis(3000)) + .setupConfig(TABLE_EXEC_ASYNC_TABLE_MAX_ATTEMPTS, 3) + .setupConfig(TABLE_EXEC_ASYNC_TABLE_BUFFER_CAPACITY, 5) + .setupTemporaryCatalogFunction("func1", FailureThenSucceedSplit.class) + .setupTableSource( + SourceTestStep.newBuilder("source_t") + .addSchema(SOURCE_SCHEMA) + // Errors on "hello#world" + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("a VARCHAR", "b VARCHAR") + .consumedBeforeRestore( + "+I[hi#there, $hi]", "+I[hi#there, $there]") + .consumedAfterRestore( + "+I[hello#world, $hello]", + "+I[hello#world, $world]", + "+I[foo#bar, $foo]", + "+I[foo#bar, $bar]", + "+I[bar#fiz, $bar]", + "+I[bar#fiz, $fiz]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT c, s FROM source_t, LATERAL TABLE(func1(c, '$')) AS T(s)") + .build(); + + /** Splitter functions. */ + public static class TableFunc1 extends AsyncTableFunction { + + private static final long serialVersionUID = -7888476374311468525L; + + public void eval(CompletableFuture> future, String str) { + if (str.contains("#")) { + List result = Arrays.stream(str.split("#")).collect(Collectors.toList()); + future.complete(result); + } else { + future.complete(Collections.emptyList()); + } + } + + public void eval(CompletableFuture> future, String str, String prefix) { + if (str.contains("#")) { + List result = + Arrays.stream(str.split("#")) + .map(s -> prefix + s) + .collect(Collectors.toList()); + future.complete(result); + } else { + future.complete(Collections.emptyList()); + } + } + } + + /** Fails then succeeds to test succeeding after restore. */ + public static class FailureThenSucceedSplit extends AsyncTableFunction { + private static final int TOTAL_FAILURES = 1; + + private final AtomicInteger calls = new AtomicInteger(0); + + public void eval(CompletableFuture> future, String str, String prefix) { + if (!str.equals("hello#world") || calls.incrementAndGet() > TOTAL_FAILURES) { + if (str.contains("#")) { + List result = + Arrays.stream(str.split("#")) + .map(s -> prefix + s) + .collect(Collectors.toList()); + future.complete(result); + } else { + future.complete(Collections.emptyList()); + } + } + throw new RuntimeException("Failure " + calls.get()); + } + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java index 1098c0585b2e5..444c17fee7e49 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.java @@ -140,6 +140,12 @@ public void testJustCall() { util.verifyRelPlan(sqlQuery); } + @Test + public void testNestedSystemCall() { + String sqlQuery = "SELECT func1(ABS(1))"; + util.verifyRelPlan(sqlQuery); + } + @Test public void testWhereCondition() { String sqlQuery = "SELECT a from MyTable where REGEXP(func2(a), 'string (2|3)')"; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java index 77164b9185afa..b98403a28466a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.java @@ -20,6 +20,8 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.functions.AsyncTableFunction; +import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram; import org.apache.flink.table.planner.plan.optimize.program.FlinkHepRuleSetProgramBuilder; import org.apache.flink.table.planner.plan.optimize.program.HEP_RULES_EXECUTION_TYPE; @@ -34,6 +36,10 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; + /** Test for {@link AsyncCorrelateSplitRule}. */ public class AsyncCorrelateSplitRuleTest extends TableTestBase { @@ -63,6 +69,8 @@ public void setup() { util.addTemporarySystemFunction("func1", new Func1()); util.addTemporarySystemFunction("tableFunc", new RandomTableFunction()); + util.addTemporarySystemFunction("scalar", new ScalarFunc()); + util.addTemporarySystemFunction("asyncTableFunc", new AsyncFunc()); } @Test @@ -82,4 +90,38 @@ public void testCorrelateIndirectOtherWay() { String sqlQuery = "select * FROM MyTable, LATERAL TABLE(tableFunc(func1(ABS(a))))"; util.verifyRelPlan(sqlQuery); } + + @Test + public void testCorrelateWithSystem() { + String sqlQuery = "select * FROM MyTable, LATERAL TABLE(asyncTableFunc(ABS(a)))"; + util.verifyRelPlan(sqlQuery); + } + + @Test + public void testCorrelateWithScalar() { + String sqlQuery = "select * FROM MyTable, LATERAL TABLE(asyncTableFunc(scalar(a)))"; + util.verifyRelPlan(sqlQuery); + } + + @Test + public void testCorrelateWithCast() { + String sqlQuery = + "select * FROM MyTable, LATERAL TABLE(asyncTableFunc(cast(cast(a as int) as int)))"; + util.verifyRelPlan(sqlQuery); + } + + /** Test function. */ + public static class AsyncFunc extends AsyncTableFunction { + + public void eval(CompletableFuture> c, Integer i) { + c.complete(Arrays.asList("blah " + i, "foo " + i)); + } + } + + /** Test function. */ + public static class ScalarFunc extends ScalarFunction { + public Integer eval(Integer param) { + return param + 10; + } + } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/AsyncCorrelateITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/AsyncCorrelateITCase.java new file mode 100644 index 0000000000000..31b9b8c71984e --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/AsyncCorrelateITCase.java @@ -0,0 +1,255 @@ +/* + * 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.planner.runtime.stream.table; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.functions.AsyncTableFunction; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.AsyncSumScalarFunction; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.AsyncTestTableFunction; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.SumScalarFunction; +import org.apache.flink.table.planner.runtime.utils.StreamingTestBase; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** IT Case tests for correlate queries using {@link AsyncTableFunction}. */ +public class AsyncCorrelateITCase extends StreamingTestBase { + + private TableEnvironment tEnv; + + @BeforeEach + public void before() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + tEnv = StreamTableEnvironment.create(env, EnvironmentSettings.inStreamingMode()); + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_BUFFER_CAPACITY, 1); + tEnv.getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_ASYNC_SCALAR_TIMEOUT, Duration.ofMinutes(1)); + } + + @Test + public void testConstantTableFunc() { + tEnv.createTemporarySystemFunction("func", new TestTableFunction()); + final List results = executeSql("SELECT * FROM func(1);"); + final List expectedRows = Arrays.asList(Row.of("blah 1"), Row.of("foo 1")); + assertThat(results).containsSequence(expectedRows); + } + + @Test + public void testConstantTableFuncNoArg() { + tEnv.createTemporarySystemFunction("func", new TestTableFunction()); + final List results = executeSql("SELECT * FROM func();"); + final List expectedRows = Arrays.asList(Row.of("blah"), Row.of("foo")); + assertThat(results).containsSequence(expectedRows); + } + + @Test + public void testConstantTableFuncWithLateral() { + tEnv.createTemporarySystemFunction("func", new TestTableFunction()); + final List results = executeSql("SELECT * FROM LATERAL TABLE(func(1));"); + final List expectedRows = Arrays.asList(Row.of("blah 1"), Row.of("foo 1")); + assertThat(results).containsSequence(expectedRows); + } + + @Test + public void testTableFunc() { + Table t1 = tEnv.fromValues(1, 2, 3).as("f1"); + tEnv.createTemporaryView("t1", t1); + tEnv.createTemporarySystemFunction("func", new TestTableFunction()); + final List results = executeSql("select * FROM t1, LATERAL TABLE(func(f1))"); + final List expectedRows = + Arrays.asList( + Row.of(1, "blah 1"), + Row.of(1, "foo 1"), + Row.of(2, "blah 2"), + Row.of(2, "foo 2"), + Row.of(3, "blah 3"), + Row.of(3, "foo 3")); + assertThat(results).containsSequence(expectedRows); + } + + @Test + public void testTableFuncRowTypeWithHints() { + Table t1 = tEnv.fromValues(1, 2, 3).as("f1"); + tEnv.createTemporaryView("t1", t1); + tEnv.createTemporarySystemFunction("func", new AsyncTestTableFunction()); + final List results = executeSql("select * FROM t1, LATERAL TABLE(func(f1))"); + final List expectedRows = + Arrays.asList( + Row.of(1, "blah 1"), + Row.of(1, "foo 1"), + Row.of(2, "blah 2"), + Row.of(2, "foo 2"), + Row.of(3, "blah 3"), + Row.of(3, "foo 3")); + assertThat(results).containsSequence(expectedRows); + } + + @Test + public void testTableFuncWithCalc() { + Table t1 = tEnv.fromValues(1, 2).as("f1"); + tEnv.createTemporaryView("t1", t1); + tEnv.createTemporarySystemFunction("func", new AsyncTestTableFunction()); + tEnv.createTemporarySystemFunction("mySum", new SumScalarFunction()); + final List results = + executeSql("select * FROM t1, LATERAL TABLE(func(mySum(f1, 10)))"); + final List expectedRows = + Arrays.asList( + Row.of(1, "blah 11"), + Row.of(1, "foo 11"), + Row.of(2, "blah 12"), + Row.of(2, "foo 12")); + assertThat(results).containsSequence(expectedRows); + } + + @Test + public void testTableFuncWithAsyncCalc() { + Table t1 = tEnv.fromValues(1, 2).as("f1"); + tEnv.createTemporaryView("t1", t1); + tEnv.createTemporarySystemFunction("func", new AsyncTestTableFunction()); + tEnv.createTemporarySystemFunction("mySum", new AsyncSumScalarFunction()); + final List results = + executeSql("select * FROM t1, LATERAL TABLE(func(mySum(ABS(f1), 10)))"); + final List expectedRows = + Arrays.asList( + Row.of(1, "blah 11"), + Row.of(1, "foo 11"), + Row.of(2, "blah 12"), + Row.of(2, "foo 12")); + assertThat(results).containsSequence(expectedRows); + } + + @Test + public void testTableFuncWithRightCalcSelectStar() { + Table t1 = tEnv.fromValues(1, 2).as("f1"); + tEnv.createTemporaryView("t1", t1); + tEnv.createTemporarySystemFunction("func", new TestTableFunction()); + final List results = + executeSql("select * FROM t1, LATERAL (SELECT * FROM TABLE(func(f1)) as T(foo))"); + final List expectedRows = + Arrays.asList( + Row.of(1, "blah 1"), + Row.of(1, "foo 1"), + Row.of(2, "blah 2"), + Row.of(2, "foo 2")); + assertThat(results).containsSequence(expectedRows); + } + + @Test + public void testTableFuncWithRightCalcWithSelect() { + Table t1 = tEnv.fromValues(1, 2).as("f1"); + tEnv.createTemporaryView("t1", t1); + tEnv.createTemporarySystemFunction("func", new TestTableFunction()); + tEnv.createTemporarySystemFunction("mySum", new AsyncSumScalarFunction()); + assertThatThrownBy( + () -> + executeSql( + "select * FROM t1, LATERAL (SELECT CONCAT(foo, 'abc') " + + "FROM TABLE(func(f1)) as T(foo))")) + .hasMessageContaining( + "Currently Async correlate does not support " + + "projections or conditions"); + } + + @Test + public void testTableFuncWithRightCalcWithConditions() { + Table t1 = tEnv.fromValues(1, 2).as("f1"); + tEnv.createTemporaryView("t1", t1); + tEnv.createTemporarySystemFunction("func", new AsyncTestTableFunction()); + assertThatThrownBy( + () -> + executeSql( + "select * FROM t1, LATERAL (SELECT * " + + "FROM TABLE(func(f1)) as T(foo) WHERE CHAR_LENGTH(foo) > 0)")) + .hasMessageContaining( + "Currently Async correlate does not support " + + "projections or conditions"); + } + + @Test + public void testFailures() { + // If there is a failure after hitting the end of the input, then it doesn't retry. Having + // the buffer = 1 triggers the end input only after completion. + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_BUFFER_CAPACITY, 1); + Table t1 = tEnv.fromValues(1).as("f1"); + tEnv.createTemporaryView("t1", t1); + AsyncErrorFunction func = new AsyncErrorFunction(2); + tEnv.createTemporarySystemFunction("func", func); + final List results = executeSql("select * FROM t1, LATERAL TABLE(func(f1))"); + final List expectedRows = Collections.singletonList(Row.of(1, "3")); + assertThat(results).containsSequence(expectedRows); + } + + private List executeSql(String sql) { + TableResult result = tEnv.executeSql(sql); + final List rows = new ArrayList<>(); + result.collect().forEachRemaining(rows::add); + return rows; + } + + /** A table function. */ + public static class TestTableFunction extends AsyncTableFunction { + + public void eval(CompletableFuture> result, Integer i) { + result.complete(Arrays.asList("blah " + i, "foo " + i)); + } + + public void eval(CompletableFuture> result) { + result.complete(Arrays.asList("blah", "foo")); + } + } + + /** A error function. */ + public static class AsyncErrorFunction extends AsyncTableFunction { + + private final int numFailures; + private final AtomicInteger failures = new AtomicInteger(0); + + public AsyncErrorFunction(int numFailures) { + this.numFailures = numFailures; + } + + public void eval(CompletableFuture> future, int ignored) { + if (failures.getAndIncrement() < numFailures) { + future.completeExceptionally(new RuntimeException("Error " + failures.get())); + return; + } + future.complete(Collections.singletonList("" + failures.get())); + } + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java index 9d7d559713b86..fcc58253c8f12 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java @@ -22,13 +22,23 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple12; import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.FunctionHint; import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.functions.AsyncScalarFunction; +import org.apache.flink.table.functions.AsyncTableFunction; +import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.types.Row; import org.apache.commons.lang3.StringUtils; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; import java.util.Random; +import java.util.concurrent.CompletableFuture; /** Test functions. */ public class JavaUserDefinedTableFunctions { @@ -115,6 +125,73 @@ public TypeInformation getResultType() { } } + /** String split table function. */ + public static class AsyncStringSplit extends AsyncTableFunction { + + public void eval(CompletableFuture> future) { + String[] strs = {"a", "b", "c"}; + for (String str : strs) { + eval(future, str); + } + } + + public void eval(CompletableFuture> future, String str) { + this.eval(future, str, ","); + } + + public void eval( + CompletableFuture> future, String str, String separatorChars) { + this.eval(future, str, separatorChars, 0); + } + + public void eval( + CompletableFuture> future, + String str, + String separatorChars, + int startIndex) { + if (str != null) { + String[] strs = StringUtils.split(str, separatorChars); + if (startIndex < 0) { + startIndex = 0; + } + List result = + new ArrayList<>(Arrays.asList(strs).subList(startIndex, strs.length)); + future.complete(result); + } + } + + public void eval(CompletableFuture> future, byte[] varbinary) { + if (varbinary != null) { + this.eval(future, new String(varbinary, StandardCharsets.UTF_8)); + } + } + } + + /** A table function. */ + @FunctionHint(output = @DataTypeHint("ROW")) + public static class AsyncTestTableFunction extends AsyncTableFunction { + + public void eval(CompletableFuture> result, Integer i) { + result.complete(Arrays.asList(Row.of("blah " + i), Row.of("foo " + i))); + } + } + + /** A sum function. */ + public static class SumScalarFunction extends ScalarFunction { + + public int eval(int a, int b) { + return a + b; + } + } + + /** A sum function. */ + public static class AsyncSumScalarFunction extends AsyncScalarFunction { + + public void eval(CompletableFuture result, int a, int b) { + result.complete(a + b); + } + } + /** Non-deterministic table function. */ public static class NonDeterministicTableFunc extends TableFunction { diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml index e82c11dfcd04d..62fe5b751af8a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCalcSplitRuleTest.xml @@ -179,6 +179,24 @@ LogicalProject(EXPR$0=[func1(1)]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml index 00b283f7b1cd9..16b7e9858072a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AsyncCorrelateSplitRuleTest.xml @@ -76,7 +76,74 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], EXPR$0=[$4]) + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-catalog-func/plan/async-correlate-catalog-func.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-catalog-func/plan/async-correlate-catalog-func.json new file mode 100644 index 0000000000000..e3308955fb916 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-catalog-func/plan/async-correlate-catalog-func.json @@ -0,0 +1,140 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "INT NOT NULL" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-async-correlate_1", + "joinType" : "INNER", + "functionCall" : { + "kind" : "CALL", + "catalogName" : "`default_catalog`.`default_database`.`func1`", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "$", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "ROW<`EXPR$0` VARCHAR(2147483647)> NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `EXPR$0` VARCHAR(2147483647)>", + "description" : "AsyncCorrelate(invocation=[func1($2, _UTF-16LE'$')], correlate=[table(func1(c,'$'))], select=[a,b,c,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])" + }, { + "id" : 3, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Calc(select=[c, EXPR$0 AS s])" + }, { + "id" : 4, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[c, s])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 3, + "target" : 4, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-catalog-func/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-catalog-func/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..52d90c43b226204a5800aa0a36dac8a4f2fd3dad GIT binary patch literal 9035 zcmeGiTWllM_1XzxcVRaHvZ(t2MQnCeR6OJHBX)=caF!G?I?ecw{D(N<`yoR8GVgAqQUs0>`$K1_~`irk0kog-mIwP%f4-rEK|Ru2?GM z=FgOJON-@)mQH5LL~%KvD|H64&b*1Unf#e-xtJ~FGWpy}wgA0ha@Mq}0&P&a#spOd zEeM=ebS79$gKKpr@Qns*a!t2|Y>P9?pn8#68r8Ln%$n})DjKw5*Gvw$N67w9QGKB| zKPNZX7Wvs0E8z;Za7tskVxOd(mY{v*oi~rZ_2et%FbU*V(*!{`xJ^R((r=xizrL`PntBQMe6LiAw8d}q#FtFdO0!@cA=Y!}RR5D}JO(HBE5f=D0t7%F zXn3gnMGplQ&4WQOrnkja;%w;#x_Q5|}G+1>NQ2s(%l&6CUTKm5&|4w{h6$?X9Rcfefr-qyjKu`IfQ`>XZjhj+a6 z7y7Fa63UTL8@4`(3l5KU=kNx8k^{-jvx#Y_PvMU4=yL=NUtzXvX${w?Zw=UVkz1Nk zMVmHYQ=kNnz%yzbpGR8_kZXtn6@wYnS!ylRsI|#Db7V|sMwQoEh;)DG5>^wO+;}!q zSU8<2JaP5vRmXA<<7yS>@y-weP2rF;>yXp?lP<|@f9PenS=VI$a60^p;@#g~JQfRs zAtB~4oEsAzb8_YI>KCu2FV6jL9)TL0$#B{yaPHlkhg$IRJoi1&=DtKqTWGBJ*avMP zfq_%Mr>E=C2fOeN{OhBSLG`+82SsQ%4vL`N4aMoafQ5jDul3vwiZiee!enLOec*1t zOW{uW`o??z9{*(RuYZ32 z{Kt3w2D;Gcz6^m`C9cLKnO5W(l|l4ZP0h&Z3Y(RZ6{Ql3(HMZ;?14k&6r761iGzp8 zEk(0w$*jOA1f`3dayNpu`w{C@N@gj!5=pDGY9ygbu?UT(qY;*lOOg~zsYynG>VLuy zBGy~bjl5m&aJAKQU!x68kj;ACGy<{e3bmO)F@7b?Z*aD3X>|yft}!R16$oW)H`sNL z16G1X_+D-O>#nz+{rkhm?|UNi?sG8a7y-igDhe~i}uw(pq}R~~)+k;mPb zJlG9_`BOYAI8vteh9Q^Nsm9A@gIScDR@p{#C?hz)?{Wr1RcGX6T2dlWNnwz|h{YpR zRTB{vQX8`g7Mqb$VBKT)0d0;w`~U|#VHwpP%dok{XdPmQ?PP}AV=>vZ6ant;Fr3v{ zof$k}6I=9mnS-0zfV|Z%h15p87?9Y*q?@h5XlTcOo!6>4gLg+bF*ZF|KrRqwaf}Vj z4bmk$5OLp?*ZSCe?t$E5$&=q;$&3F7;q6Mg;zp2M_JM3+7=hfW2;>M^?%C2?ztQrh zQ4O3_-Oa88YVjJdg1HBfcD1^2?sP2EJ8k^RGrDvSCw`6sQr7aSlXVvs+K23)GOoRb|Qw+3$6oqVO>Qw8#CnW@vUT*i&p)>}DN%d^E+T z-C%}-U1O>V9veLDZ5pypA!n{YQnf=H9>aa}#vU7D2UPUMjdL(1i6ulJwIvNAgrffu z0#1MRnUD6o_VdXbwy+N%m$);eE(N)?zY*i%dgoBRbWd=BM8{>6@CH zgd{TWC?Vq{G&wolyHr1}gLF?r*s=!`$O4is@g5-{-9npgp%2}T2;py#", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 19, + "type" : "stream-exec-async-correlate_1", + "joinType" : "INNER", + "functionCall" : { + "kind" : "CALL", + "catalogName" : "`default_catalog`.`default_database`.`func1`", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "$", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "ROW<`EXPR$0` VARCHAR(2147483647)> NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `EXPR$0` VARCHAR(2147483647)>", + "description" : "AsyncCorrelate(invocation=[func1($2, _UTF-16LE'$')], correlate=[table(func1(c,'$'))], select=[a,b,c,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])" + }, { + "id" : 20, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Calc(select=[c, EXPR$0 AS s])" + }, { + "id" : 21, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[c, s])" + } ], + "edges" : [ { + "source" : 18, + "target" : 19, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 19, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 20, + "target" : 21, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-exception/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-exception/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..87c5501e17e05fb81ac970d19c7007e66d80d539 GIT binary patch literal 8896 zcmeGiU5Fc1_fB@dwN=|)broBGSm;&|o0%k=WG5DcWSh3ZxVuf#R<>ZgnYoipJDJIM z=Vmusp$?Q+C=^BPD`$Ldy&@;$uB!m=?KHS;SN^91A*KMN275su5pK zol2#atXPfe))ckfq#oB|@YGvMMLF5FO%|U{#S@A6R4SH9O!2N{a!PUWB566M&7uZL zB-5#ACY{2%q1LqNOhUo2c*=;SaWsR|YBZTpYf;sR#iDV|(2~ho0%NR#WZ%GVWL>%W z+(0XH^Z8P?GFPgWE7?lEdZ19QlnS|{mBQR?^`5x{c{Em@FBU4R9eFWs>{zyVG+!;} zONDH)u#hhSG=iu@>oRU(twChNGVN3FVal4*Y&y2gmRqFFOv{yP*d;Qi6feuUWk%+y z*)*BCNU|JMCe*~1d79A4i*=T5+;STZ1HMD>>w2Dk@rKWie7~#ieD}^H9~~*2hfd)H z9BGY7Z>nN<+19E{*ko0wMJQ$tt-9Q{)e!o~T@u=2Bd2jZ4X$ZKGqKuKG#SfGN7aOC zL{+_(NoO)Cl1iq*z7K$u!pMIzlgnZ5dXKqXMhR}3c3q}zo0&}_dk{)Ahw5@!bmuM7 zBsL2fdL4L|jdz^mi!g%5kN@tQQ9_j-0T`}XWQF;v%axp^1DN3u&mKq@H}I;LzeWFX zDN@$q!D+j)<CKTH)JyB^v#x<9+NjM+<|3ho{v+V#~% zbD@XkdK(;Up|I_)+2i^FyXZq^6f9nZVnGwOF7X8KtKe$_ehp$-&YmTQTDs`({`2(E z$cfzEuRi|z{nO{R-)ZfG-u_V+_HY++gD548VvWJuZClLc)O?QP$34>H51^}k)E<=Q zeU$gMJThywn3hanBT7;@6-~yIS`^2p)u^fgcsheK(>g|I{2T}fk$L^uUXGA_Y{L$d z?=V7bY=!)=7u4k}UIdo>o#~EsIEfG>A2f-jyWAc5ru5TmFW>yigU?qZNGhV82JZ{H zT!zMqJe(*zH&lwI3vr5~y%(h(41WK~AIG1IJk*S!ND+-{j=cz}x9Qk37IuN~H6CFe zrxXF&Uqn~=!I6u_P~wsUDFo3)igx#k2lQMSk_yi)(NF{p12-|(-101L0=Lo7MDM|^ z{U^|<-#_oTkO4b3L)V=M2|ZUkNi+sh51%q^ouP?7;&Q$U@Vy%_0CQ){ClxT=6=GV1 zWk>YG*llq3off1w8w<}se@$>Hq7fmx`(JbhNq@fxvn9cLs5EyRba?F$wR3d`0_?4~g#(IKEC?jhZdbsjm(Bv=@`OPus94xCU;5HYd&7!<`mnkP-$d zHYfGrM0ZGM_-l-za7rLNcQ@x&6U|#Cjx@oug${4+63`pFG;{GyFUy>!sRf(U?nleF z|9W~)f*%Apy$I|ZW5OIFarX4^!uKz!r}w{}<4CQwVuM0EH4}am2Pq)2v@A#XdhH@py*u>&ek)c#?=?|_{i9$L=gKgWfl@&qu z7KZv(V8)?V$X1Pz6?H3y4)?O^H|wz6laQVEPy$`RjoDO>61d#*irjJ^xdVUN$Zg~H zne6P`Y+)u_go3Yc;fWQRAONUi%|9jdamAGZ+zNeX6PAgWWVS;!seH6NpPxPO2e|*? At^fc4 literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-join-filter/plan/async-correlate-join-filter.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-join-filter/plan/async-correlate-join-filter.json new file mode 100644 index 0000000000000..079e78a9d89bd --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-join-filter/plan/async-correlate-join-filter.json @@ -0,0 +1,207 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 9, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "INT NOT NULL" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t, filter=[]]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 10, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$LIKE$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "%hello%", + "type" : "CHAR(7) NOT NULL" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$LIKE$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "%fiz%", + "type" : "CHAR(5) NOT NULL" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>", + "description" : "Calc(select=[a, b, c], where=[(LIKE(c, '%hello%') OR LIKE(c, '%fiz%'))])" + }, { + "id" : 11, + "type" : "stream-exec-async-correlate_1", + "joinType" : "INNER", + "functionCall" : { + "kind" : "CALL", + "catalogName" : "`default_catalog`.`default_database`.`func1`", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "ROW<`EXPR$0` VARCHAR(2147483647)> NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `EXPR$0` VARCHAR(2147483647)>", + "description" : "AsyncCorrelate(invocation=[func1($2)], correlate=[table(func1(c))], select=[a,b,c,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])" + }, { + "id" : 12, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Calc(select=[c, EXPR$0 AS s])" + }, { + "id" : 13, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[c, s])" + } ], + "edges" : [ { + "source" : 9, + "target" : 10, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 11, + "target" : 12, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 12, + "target" : 13, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-join-filter/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-join-filter/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..5e27e6de51e9eddfe84db1c0d2290b623443ce99 GIT binary patch literal 8907 zcmeHNZ;Tto6(9SK%O6PYNUl|rlqeG6pNMz;Z*3ouD%lr@SeUz$?Ih@^Fyq~^y}9-7 zc6WT|%ZD^+gs4=hQ1weI1my!C3W9)ykl-^25Fmu$BSMHzK@~^{kg8U#()Y&eyY;Ov zb*kh#q>_v)DJ31v%p}sXOP85# z=_ZfFV@g`7M`lr4w>2tuU=#Peh_xtsY4xV)4jKG@gklnKV#jlw^%ELMDC& zs!a?r032?J9GYLqm$H@lQng&kR`S(jg>t1-$epYd=I5#p&L7K@sq#XxP+4!ti+)q5 zvc;46YB^sjWQ&Exdka2Wr>HaAq)xy8gbHNBU}l3i|4;pJ+o^16_C&Vy%PUu|__mEPvJh}dYU`3} zwOW=b;S%cZ88zxMsc4xE->%XB*|%lPT?1EecOTQfuRflO0iK&}aIl5Kw!2}E8&uiH zzqfL|^1vHMezk34=NFHk|NK(vLg5{##&hF592~J)Z6>Z};?Z<86{ReeNoS&3oiZh! zfrCv=1KBMeITYgAaKheNw%U%$N{qtMS;m9aquSzgmDW|3rZf^$m6=E~N#l`BEyW_~ zS_(#@s;PPs75fYP#JM_9?Hh5LI)S;~+$*W$&{Zk7%GrXWw{)&Avn*Emoa$Ryx^nQDOO>}iZh{?z58!0=%_tnjE}Z!hhjE9h97?xsa2M&|?89gJv{zmv zw|Jxhl9xS_Pq!LhF(p%H>3B2}o6%?_sm9Y0T2m=tu~>A5CR3?|RwLv`+dx0?gUttf zBV6HBz!f%I_W2HHj!6yQ#0RVTrn`%yLf-ju?~`D@?*gGO1eTGC*tH zGG`6y0^?h|j1g_fqUvao-0VRdREv}NHNzo*Ldy=>A5;(c39#7pN-`EE;~-6pHMcrP zTOe)HH_!*Eee_{6>9sFdE-XdM+vPs)HpdbKwXq810;O_D1dUe z?Xl_!-RBcpgjL716Vz>@_>K)AbvTd({Lx_E*Z`rzJWdGR-CvM%B=p0>30nfW#!B<2 z!G?DVP>cR6&02;5QwCG%9zJQdEKRTL0)&HK|6}rw*lSANlJ+!6u|~@?+l&uakABckeo9vCZT9>RkBvwm-f@pB^Is^u}F?qdGRsYMm_>0byW-K5-bzpWKk2i~)i=E? zw_3W|+no0Qx_r;y&)y!#a1Qx}Vc(b%(;>p2t{tBJ=DEz-qp#+W$lj=i_`VbS-m|f( zgJw<&V_grTp6%As- zVX}RZOli8SQb(JmrV4gu+9KKIRUM~ru@B#qy>H#r-KM{$+^)Ip7S{CGl-sV&Obx3h z>K5EkVlCPV}x?=KMJt> zvyXqW{keyxUk?+= ze0>tKB74TkiJ+)KqxS247%*~x6W9W7UlRc%px)A&-qHZQg$d!$FgMpbX0vnibA{P# t5z@}y3?EwK2?~HYHvGrK09Rb=fU_`gG=7~hF|-46TIG}Fh5X#He*tK6=#c;b literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-left-join/plan/async-correlate-left-join.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-left-join/plan/async-correlate-left-join.json new file mode 100644 index 0000000000000..88f1afb724ac6 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-left-join/plan/async-correlate-left-join.json @@ -0,0 +1,136 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 14, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "INT NOT NULL" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 15, + "type" : "stream-exec-async-correlate_1", + "joinType" : "LEFT", + "functionCall" : { + "kind" : "CALL", + "catalogName" : "`default_catalog`.`default_database`.`func1`", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "ROW<`EXPR$0` VARCHAR(2147483647)> NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `EXPR$0` VARCHAR(2147483647)>", + "description" : "AsyncCorrelate(invocation=[func1($2)], correlate=[table(func1(c))], select=[a,b,c,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[LEFT])" + }, { + "id" : 16, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Calc(select=[c, EXPR$0 AS s])" + }, { + "id" : 17, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[c, s])" + } ], + "edges" : [ { + "source" : 14, + "target" : 15, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 16, + "target" : 17, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-left-join/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-left-join/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..519209188f39ce4bd08d382024029d13fb90dc12 GIT binary patch literal 9035 zcmeGiU5pz?dF(qbN!#Qu%>|K!R)>;QRPFxwf6XB(-^JHhb?;8LlT;T8Gu|EBn_KUi z-SM3-ppb$hcq$SSYAb%)hrS@CQV~KzJn#UheJD>*@l*-%(n<)VNT5~}Q24&}?s-09bJ7O2rMhKk%yuL-C2cD8*j8#imD)7qnyncL+jNpzOqHQD+EPWj z)-iP+OC(dNxDuC>DOt(J64_WfE;)3AwJqJ`f~Lt#$p~qxXmUnY_#YDcUoKHs&;`8~1wgWvr2UuS<9{YEQFqD3;TTIL4mUboCegF1ltcn`5b&4%PQ%@@go zhSktjY9Lvh#EUSG)RULw%1d(PCAnNk9;{UCkD2ebH*9h`OrFbj$=U=FVVQ`MNtn&$ zbzy6nwqUmD0He>+_WV^c?e(u(j?RH_PEK465WP~n##9bePp<2x#>w0ea&cY>=6wbr z0PZPP7WU)|?_ z`^d|0(H~5ZNP$c`u=GJ(uz75{n>X;28cA-RO`L;z30HJqpA%sC8go=zZ}*J);ebt- zxviTGv}qF-1xnxyJfp_(d9vFExke~ZF`7YLruJf!+B>YXK&A|4Hh8m(NRLKtU^TBF)Yj^M7buITa&Q^0DAB`Z;9Co>*PPwh08j#H1n_iAvEnW3Dr_Mz4MxKef}p42-Mz5hWq{)j(vP*R~HV?EB^x8dWT457mbCt zeb5#X7})hgEd!fA*oC*@-yeO9s@DVCC?W^3Q3UO7EKc_T76Tf-*L^oA&cHf|lC_c7 zL2vo(SPH`$2Uf#=pzW3%G`K|Y~=b+mi0LY)wtq<%FzeSUR0dDFFJA2ks-qp;+9|d4wDHJH- zx`EO6kllDHJyWI5mZ?@Zsm`lbo7t3GcGW@MR$Y__u3SZBYFvqFLcCVb3Q4M_1tp^< z1R96jMM9O6wRi&b`URlQRqn@dupeq}gw*VCo6#1;4!4sTy%w9PmaU0!cZcr0!CK7Z z0R`{UKA_uPW&`q82NY79@M1txh{;~I2BV>U|8+rc6ihxC;l$YVXaTuEn8mYS2;c8V z#Qi{y^{Iuz(}m@VC%@5>7yk>wJCJn6A&^}D>3nG%f!wbM%a!Jd=FT`+yh7l-Ue{)cP!J}ZT!kJIwi!3pQC`3wYu(R-NnW2$5k5-Os+Hc^8vsW zjgrG_WJc2+mD<`OHC1p`nYKobdQD3dJ_dmIY5!d}b*C9@DR*j4r-dy(nsUc!GgHH+ zsk#O37d#y4n5sb`XRbj~wND#v!$b4N9vf00RPx1*V=yI&B|;#z6&pneW&a}t9RBJz ze|Pw`=VpHvC6HWu0BCGc_e9_%70!^tdB`NzWh#V=EK}M>w6s0UPrq?8G&MI1No3wv zLe5QSa&j>|)iADubf1H;We5|<0+KF?kPwh=ahq;&2;Hs-;ct}W?2*OX^2%~yF;|2y q0Xz-QZQ}$2KpZ>1l^9}*+XGMwL%9jkgt@vSkY+4jDzE03FZ>%%S^A9t literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-system-func/plan/async-correlate-system-func.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-system-func/plan/async-correlate-system-func.json new file mode 100644 index 0000000000000..1f7d790918de8 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-system-func/plan/async-correlate-system-func.json @@ -0,0 +1,140 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 5, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "INT NOT NULL" + }, { + "name" : "c", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, source_t]], fields=[a, b, c])", + "inputProperties" : [ ] + }, { + "id" : 6, + "type" : "stream-exec-async-correlate_1", + "joinType" : "INNER", + "functionCall" : { + "kind" : "CALL", + "systemName" : "STRING_SPLIT", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "#", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "ROW<`EXPR$0` VARCHAR(2147483647)> NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `EXPR$0` VARCHAR(2147483647)>", + "description" : "AsyncCorrelate(invocation=[STRING_SPLIT($2, _UTF-16LE'#')], correlate=[table(STRING_SPLIT(c,'#'))], select=[a,b,c,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])" + }, { + "id" : 7, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Calc(select=[c, EXPR$0 AS s])" + }, { + "id" : 8, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b", + "dataType" : "VARCHAR(2147483647)" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[c, s])" + } ], + "edges" : [ { + "source" : 5, + "target" : 6, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 6, + "target" : 7, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 7, + "target" : 8, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-system-func/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-correlate_1/async-correlate-system-func/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..54851ba7d57811d7fb803ae2df1b23f31e8d8a23 GIT binary patch literal 9035 zcmeGiZHya7b=G%Wb8V8lHdjOvs7^{6i0JN~Ki2kf63BL(#=^ZjY+r&d5@x(Rwl}xl zHM`?GUqDF;sz3r22~|S;Xwx69_|r<25LN&67avj)Ki~&H_{Se41c(Zxs)#DQH{Lz3 z&(76By%Qj>WY5mbn>TOXd-LYadt3NwLneew!w(;+SCHx53%~z#HTl+ePv_o$Rr9@SDzOBv}}Qe8^KwWMB; z$d7K;XV0O4V{DQ^{8^ z70Q)TA-hp2tgTj`S-X@c^X2toq0;Zji*fT;GsTU3wVW>%GR4AXz67lia@BDg3T;!p z$&|VUQcyUpSxj*|HaA;L;k#|t;ily&`7UR!O|3F>O=_7pnR|Y?sch4>*K|1Go+2my zh~kr8`wRJ)=aSFfXC|D%Q!bm#GQ3Na(+br8@y2VPeEr$G)d-P_AK{}1F;8#fByZyFT51_Rx3gxMKY&5_BP1gbnKi(JwW_u53xn=*HUaQ?;ItQw!w@lmMWN`$!IIjxx zei0x5@?_gb6-;_MG-(kOf*Dt+*Tm_&Z4l#A!z&RA(-Wi500rtLY(X)i)@mIO>ebhSvYl(9K*>=j>9=gj#x1Q-CsaB4wgo4xg_zRB;i<} zEbcbKA5ueVIJMVnuy(a`3N$2F_Aqn@vdg49R$DDCpx{Ld( z`|NK%^`l?V7pI6+Ak!XfeP92#U9rrkiDwqa8s2hPGfN*v#(yKUgBM1hL&6zVEG1a zo|A{JW=gp$nbPfh_wEVJJ%h8=#Oo6h0xe>f7q*wDJXXpQwi-_0kMf}6(82s>t+BC67zS%U-@g{RIP_u{)_yjP1$8#Z= z&-5S&c?4br9%fk%Ah<@(0EbMoxmsds<* zrNRrpmfwdabh_`rPrQgy#68K`D)n}4y}Cn9UUk~crQC6=9;cjD5e(puI1-)KQfWq0 zOR^D-#~>1&O37&>X2>Z$nT{J;I+aL9Aqf3>pba9?(ub|!Fm4--p&OaUT}E3F1KJBp z_gY-0JFcO?9Spkj7Hct^hr$&1X&<kGh$drGVSBDnai@IY}tM7+o%|^lI zgOMtXF^(5l30;?kvVlAy?0pE4(<3p@Gugt^h1H7Bzwv_5{|n4JlCbkbz`6X>`O+kk zbXbw3DY8DarD@w#bf?vF>`)f1Mm+|REV~i1RD;z41GV}9SV7%GNC#>II1f7}NTW z`{dnMv%k9YjdN?iyZyn1A488&0)z-e+J_czzOhOVEg;BIK+0O*5?Oa8xA(g4qNBx4 zCO#hkM3D$Nwn^p<)6=PI8%AOyOeysLus;(>A?kxTV~wd7T!v1Zc_~ zugz=&n`Y_`C?LGdc5L0EkTW+RsoJLvkMEIrV}BWHA5;xEh+{A%iH}4ewG|zQ3FY7; z1RVbIw|{%=l`k*66(NvZdl+bJQQ=8&q77$A!H;4p>oOg}MUJiRA-cRbEJ(j`GBP!{ z07+!t7eYoPG^Jk_XGhv_~KVap**zzaya#D|!GbSrywD.", classOf[ValidationException] ) @@ -262,12 +259,9 @@ class LookupJoinTest extends TableTestBase with Serializable { expectExceptionThrown( "SELECT * FROM T JOIN LookupTable8 " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b = D.name AND T.ts = D.ts", - "Could not find an implementation method 'eval' in class " + + "Method 'eval' of function class " + "'org.apache.flink.table.planner.plan.utils.InvalidAsyncTableFunctionEvalSignature2' " + - "for function 'default_catalog.default_database.LookupTable8' that matches the " + - "following signature:\nvoid eval(java.util.concurrent.CompletableFuture, " + - "java.lang.Integer, java.lang.String, " + - "java.time.LocalDateTime)", + "must have a first argument of type java.util.concurrent.CompletableFuture.", classOf[ValidationException] ) @@ -281,12 +275,19 @@ class LookupJoinTest extends TableTestBase with Serializable { expectExceptionThrown( "SELECT * FROM T JOIN LookupTable10 " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b = D.name AND T.ts = D.ts", - "Could not find an implementation method 'eval' in class " + + "Method 'eval' of function class " + "'org.apache.flink.table.planner.plan.utils.InvalidAsyncTableFunctionEvalSignature3' " + - "for function 'default_catalog.default_database.LookupTable10' that matches the " + - "following signature:\n" + - "void eval(java.util.concurrent.CompletableFuture, java.lang.Integer, " + - "org.apache.flink.table.data.StringData, org.apache.flink.table.data.TimestampData)", + "must have a first argument of type java.util.concurrent.CompletableFuture.", + classOf[ValidationException] + ) + + createLookupTable("LookupTable11", new InvalidAsyncTableFunctionType) + expectExceptionThrown( + "SELECT * FROM T JOIN LookupTable11 " + + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b = D.name AND T.ts = D.ts", + "Method 'eval' of function class " + + "'org.apache.flink.table.planner.plan.utils.InvalidAsyncTableFunctionType' " + + "must have a first argument of type java.util.concurrent.CompletableFuture.", classOf[ValidationException] ) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/lookupFunctions.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/lookupFunctions.scala index 663231b3d7f4b..02882a1df05fb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/lookupFunctions.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/lookupFunctions.scala @@ -66,6 +66,11 @@ class InvalidAsyncTableFunctionEvalSignature3 extends AsyncTableFunction[RowData def eval(resultFuture: ResultFuture[RowData], a: Integer, b: StringData, c: JLong): Unit = {} } +@SerialVersionUID(1L) +class InvalidAsyncTableFunctionType extends AsyncTableFunction[String] { + def eval(resultFuture: ResultFuture[String], a: Integer, b: StringData, c: JLong): Unit = {} +} + @SerialVersionUID(1L) class AsyncTableFunctionWithRowData extends AsyncTableFunction[RowData] { def eval( @@ -84,7 +89,7 @@ class AsyncTableFunctionWithRowDataVarArg extends AsyncTableFunction[RowData] { @SerialVersionUID(1L) class AsyncTableFunctionWithRow extends AsyncTableFunction[Row] { @varargs - def eval(obj: AnyRef*): Unit = {} + def eval(resultFuture: CompletableFuture[JCollection[Row]], obj: AnyRef*): Unit = {} } @SerialVersionUID(1L) diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/calc/async/AsyncFunctionRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/calc/async/AsyncFunctionRunner.java index 5687572fd9bad..456f07bcd962c 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/calc/async/AsyncFunctionRunner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/calc/async/AsyncFunctionRunner.java @@ -32,7 +32,7 @@ */ public class AsyncFunctionRunner extends RichAsyncFunction { - private static final long serialVersionUID = -6664660022391632123L; + private static final long serialVersionUID = -7198305381139008806L; private final GeneratedFunction> generatedFetcher; diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/calc/async/DelegatingAsyncResultFuture.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/calc/async/DelegatingAsyncResultFuture.java index 6c071686dde4a..1421be97ce9f9 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/calc/async/DelegatingAsyncResultFuture.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/calc/async/DelegatingAsyncResultFuture.java @@ -32,7 +32,8 @@ import java.util.function.BiConsumer; /** - * Inspired by {@link org.apache.flink.table.runtime.operators.join.lookup.DelegatingResultFuture} + * Inspired by {@link + * org.apache.flink.table.runtime.operators.correlate.async.DelegatingAsyncTableResultFuture} but * for {@link org.apache.flink.table.functions.AsyncScalarFunction}. */ public class DelegatingAsyncResultFuture implements BiConsumer { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/AsyncCorrelateRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/AsyncCorrelateRunner.java new file mode 100644 index 0000000000000..00811547214a6 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/AsyncCorrelateRunner.java @@ -0,0 +1,159 @@ +/* + * 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.runtime.operators.correlate.async; + +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.util.FunctionUtils; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.streaming.api.functions.async.CollectionSupplier; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.utils.JoinedRowData; +import org.apache.flink.table.runtime.generated.GeneratedFunction; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * Async function runner for {@link org.apache.flink.table.functions.AsyncTableFunction}. It invokes + * the UDF for each of the input rows, joining the responses with the input. + */ +public class AsyncCorrelateRunner extends RichAsyncFunction { + + private static final long serialVersionUID = 7004349814516992850L; + + private final GeneratedFunction> generatedFetcher; + private final DataStructureConverter fetcherConverter; + private transient AsyncFunction fetcher; + + public AsyncCorrelateRunner( + GeneratedFunction> generatedFetcher, + DataStructureConverter fetcherConverter) { + this.generatedFetcher = generatedFetcher; + this.fetcherConverter = fetcherConverter; + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + + ClassLoader cl = getRuntimeContext().getUserCodeClassLoader(); + this.fetcher = generatedFetcher.newInstance(cl); + + FunctionUtils.setFunctionRuntimeContext(fetcher, getRuntimeContext()); + FunctionUtils.openFunction(fetcher, openContext); + + fetcherConverter.open(cl); + } + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) throws Exception { + try { + JoinedRowResultFuture outResultFuture = + new JoinedRowResultFuture(input, resultFuture, fetcherConverter); + + fetcher.asyncInvoke(input, outResultFuture); + } catch (Throwable t) { + resultFuture.completeExceptionally(t); + } + } + + @Override + public void close() throws Exception { + super.close(); + FunctionUtils.closeFunction(fetcher); + } + + private static final class JoinedRowResultFuture implements ResultFuture { + private final DataStructureConverter resultConverter; + + private RowData leftRow; + private ResultFuture realOutput; + + private JoinedRowResultFuture( + RowData row, + ResultFuture resultFuture, + DataStructureConverter resultConverter) { + this.leftRow = row; + this.realOutput = resultFuture; + this.resultConverter = resultConverter; + } + + @Override + public void complete(Collection result) { + try { + Collection rightRows = wrapPrimitivesAndConvert(result); + completeResultFuture(rightRows); + } catch (Throwable t) { + realOutput.completeExceptionally(t); + } + } + + private void completeResultFuture(Collection rightRows) { + realOutput.complete( + () -> { + if (rightRows == null || rightRows.isEmpty()) { + return Collections.emptyList(); + } else { + List outRows = new ArrayList<>(); + for (RowData rightRow : rightRows) { + RowData outRow = + new JoinedRowData(leftRow.getRowKind(), leftRow, rightRow); + outRows.add(outRow); + } + return outRows; + } + }); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private Collection wrapPrimitivesAndConvert(Collection result) { + Collection rowDataCollection; + if (resultConverter.isIdentityConversion()) { + rowDataCollection = (Collection) result; + } else { + rowDataCollection = new ArrayList<>(result.size()); + for (Object element : result) { + if (element instanceof RowData) { + rowDataCollection.add((RowData) element); + } else { + rowDataCollection.add(resultConverter.toInternal(element)); + } + } + } + return rowDataCollection; + } + + @Override + public void completeExceptionally(Throwable error) { + realOutput.completeExceptionally(error); + } + + @Override + public void complete(CollectionSupplier supplier) { + throw new UnsupportedOperationException(); + } + + public void close() throws Exception {} + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/DelegatingResultFuture.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFuture.java similarity index 55% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/DelegatingResultFuture.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFuture.java index 6bbf0a89cff6e..c95a51d786545 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/DelegatingResultFuture.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFuture.java @@ -16,11 +16,15 @@ * limitations under the License. */ -package org.apache.flink.table.runtime.operators.join.lookup; +package org.apache.flink.table.runtime.operators.correlate.async; import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.types.Row; +import java.util.ArrayList; import java.util.Collection; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; @@ -29,27 +33,46 @@ * is used as a bridge between {@link org.apache.flink.table.functions.AsyncTableFunction} and * {@link org.apache.flink.streaming.api.functions.async.AsyncFunction}. */ -public class DelegatingResultFuture implements BiConsumer, Throwable> { +public class DelegatingAsyncTableResultFuture implements BiConsumer, Throwable> { - private final ResultFuture delegatedResultFuture; - private final CompletableFuture> completableFuture; + private final ResultFuture delegatedResultFuture; + private final boolean needsWrapping; + private final boolean isInternalResultType; - public DelegatingResultFuture(ResultFuture delegatedResultFuture) { + private final CompletableFuture> completableFuture; + + public DelegatingAsyncTableResultFuture( + ResultFuture delegatedResultFuture, + boolean needsWrapping, + boolean isInternalResultType) { this.delegatedResultFuture = delegatedResultFuture; + this.needsWrapping = needsWrapping; + this.isInternalResultType = isInternalResultType; this.completableFuture = new CompletableFuture<>(); this.completableFuture.whenComplete(this); } @Override - public void accept(Collection outs, Throwable throwable) { + public void accept(Collection outs, Throwable throwable) { if (throwable != null) { delegatedResultFuture.completeExceptionally(throwable); } else { + if (needsWrapping) { + List wrapped = new ArrayList<>(); + for (Object value : outs) { + if (isInternalResultType) { + wrapped.add(GenericRowData.of(value)); + } else { + wrapped.add(Row.of(value)); + } + } + outs = wrapped; + } delegatedResultFuture.complete(outs); } } - public CompletableFuture> getCompletableFuture() { + public CompletableFuture> getCompletableFuture() { return completableFuture; } } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/correlate/AsyncCorrelateRunnerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/correlate/AsyncCorrelateRunnerTest.java new file mode 100644 index 0000000000000..2c1252bb0878e --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/correlate/AsyncCorrelateRunnerTest.java @@ -0,0 +1,195 @@ +/* + * 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.runtime.operators.aggregate.correlate; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.TaskInfoImpl; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.functions.util.RuntimeUDFContext; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.streaming.api.functions.async.CollectionSupplier; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.data.utils.JoinedRowData; +import org.apache.flink.table.runtime.generated.GeneratedFunctionWrapper; +import org.apache.flink.table.runtime.operators.correlate.async.AsyncCorrelateRunner; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests {@link AsyncCorrelateRunner}. */ +public class AsyncCorrelateRunnerTest { + + @Test + public void testRows() throws Exception { + TestResultFuture resultFuture = new TestResultFuture(); + AsyncCorrelateRunner runner = + new AsyncCorrelateRunner( + new GeneratedFunctionWrapper<>(new ImmediateCallbackFunction()), + createConverter(RowType.of(new IntType()))); + runner.setRuntimeContext(createRuntimeContext()); + runner.open((OpenContext) null); + runner.asyncInvoke(GenericRowData.of(0), resultFuture); + Collection rows = resultFuture.getResult().get(); + assertThat(rows).containsExactly(); + + resultFuture = new TestResultFuture(); + runner.asyncInvoke(GenericRowData.of(1), resultFuture); + rows = resultFuture.getResult().get(); + assertThat(rows) + .containsExactly(new JoinedRowData(GenericRowData.of(1), GenericRowData.of(1))); + + resultFuture = new TestResultFuture(); + runner.asyncInvoke(GenericRowData.of(2), resultFuture); + rows = resultFuture.getResult().get(); + assertThat(rows) + .containsExactly( + new JoinedRowData(GenericRowData.of(2), GenericRowData.of(10)), + new JoinedRowData(GenericRowData.of(2), GenericRowData.of(20))); + } + + @Test + public void testException() throws Exception { + TestResultFuture resultFuture = new TestResultFuture(); + AsyncCorrelateRunner runner = + new AsyncCorrelateRunner( + new GeneratedFunctionWrapper<>(new ExceptionFunction()), + createConverter(RowType.of(new IntType()))); + runner.setRuntimeContext(createRuntimeContext()); + runner.open((OpenContext) null); + runner.asyncInvoke(GenericRowData.of(0), resultFuture); + + assertThatThrownBy(() -> resultFuture.getResult().get()) + .isInstanceOf(ExecutionException.class) + .cause() + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Error!!!"); + + TestResultFuture otherResultFuture = new TestResultFuture(); + runner.asyncInvoke(GenericRowData.of(1), otherResultFuture); + assertThatThrownBy(() -> otherResultFuture.getResult().get()) + .isInstanceOf(ExecutionException.class) + .cause() + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Other Error!"); + } + + private RuntimeContext createRuntimeContext() { + return new RuntimeUDFContext( + new TaskInfoImpl("", 1, 0, 1, 0), + getClass().getClassLoader(), + new ExecutionConfig(), + new HashMap<>(), + new HashMap<>(), + UnregisteredMetricsGroup.createOperatorMetricGroup()); + } + + private DataStructureConverter createConverter(LogicalType logicalType) { + return cast( + DataStructureConverters.getConverter( + TypeConversions.fromLogicalToDataType(logicalType))); + } + + @SuppressWarnings("unchecked") + private DataStructureConverter cast( + DataStructureConverter converter) { + return (DataStructureConverter) (Object) converter; + } + + /** Test Function. */ + public static class ImmediateCallbackFunction implements AsyncFunction { + public ImmediateCallbackFunction() {} + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) throws Exception { + List result = new ArrayList<>(); + int val = input.getInt(0); + if (val == 1) { + result.add(Row.of(1)); + } else if (val > 1) { + result.add(Row.of(10)); + result.add(Row.of(20)); + } + resultFuture.complete(result); + } + } + + /** Error Function. */ + public static class ExceptionFunction implements AsyncFunction { + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) throws Exception { + int val = input.getInt(0); + + if (val == 0) { + throw new RuntimeException("Error!!!!"); + } else { + resultFuture.completeExceptionally(new RuntimeException("Other Error!")); + } + } + } + + /** Test result future. */ + public static final class TestResultFuture implements ResultFuture { + + CompletableFuture> data = new CompletableFuture<>(); + + @Override + public void complete(Collection result) { + data.complete(result); + } + + @Override + public void completeExceptionally(Throwable error) { + data.completeExceptionally(error); + } + + public CompletableFuture> getResult() { + return data; + } + + @Override + public void complete(CollectionSupplier supplier) { + try { + data.complete(supplier.get()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } +}