-
Notifications
You must be signed in to change notification settings - Fork 13.7k
[FLINK-37724] Adds AsyncTableFunction as a fully supported UDF type #26567
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
4723794
68051e4
fe82fdc
5cc70ac
574850e
54b1276
bc25fe4
7e0e907
99da3bd
12d659b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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> 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<Integer> list) {} | ||
|
||
public void m2(List list) {} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -422,7 +422,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<Duration> TABLE_EXEC_ASYNC_SCALAR_TIMEOUT = | ||
|
@@ -456,6 +456,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<Integer> TABLE_EXEC_ASYNC_TABLE_MAX_CONCURRENT_OPERATIONS = | ||
key("table.exec.async-table.max-concurrent-operations") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Looks we changed the naming here. @becketqin , should we change in https://github.com/apache/flink/blob/master/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java#L465 as well to make it consistent? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, I think that makes sense. BTW, if we have a FLIP, we should update the FLIP to make it aligned with the code. Admittedly, changing the config name here causes the inconsistent config name in Another thing is that at this point, our design principle for configs in SQL seems not consistent. Some of the configs are set using hints, while others are in the execution config. It would be good to have a common design principle, so users don't need to guess how to config the physical behaviors. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Should I do that as part of this PR or have a followup for
I agree. It's a bit inconsistent. I think all async operations (scalar, table, lookup joins, ml_predict) support the execution config, though only lookup joins at the moment also support hints. Hints support could certainly be added in the future to others. I can file jiras for each feature to add hint support, if it can be retroactively added to old FLIPs. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks Allan.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I asked in FLIP-525 voting thread to make the config There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Ok, that all sounds good. I'll do that.
I agree, that seems good to have a single FLIP if the aim is to make them all consistent. |
||
.intType() | ||
.defaultValue(10) | ||
.withDescription( | ||
"The max number of concurrent async i/o operations that the async table function can trigger."); | ||
|
||
@Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) | ||
public static final ConfigOption<Duration> 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<RetryStrategy> 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<Duration> 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<Integer> TABLE_EXEC_ASYNC_TABLE_MAX_RETRIES = | ||
key("table.exec.async-table.max-retries") | ||
.intType() | ||
.defaultValue(3) | ||
.withDescription( | ||
"The max number of async retry attempts to make before task " | ||
+ "execution is failed."); | ||
|
||
// ------------------------------------------------------------------------ | ||
// Async ML_PREDICT Options | ||
// ------------------------------------------------------------------------ | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -35,6 +35,7 @@ | |
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.Map; | ||
|
@@ -43,6 +44,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 +210,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( | ||
becketqin marked this conversation as resolved.
Show resolved
Hide resolved
|
||
Class<?> baseClass, boolean verifyFutureContainsCollection) { | ||
return (method, state, arguments, result) -> { | ||
checkNoState(state); | ||
checkScalarArgumentsOnly(arguments); | ||
|
@@ -220,12 +223,29 @@ static MethodVerification createParameterAndCompletableFutureVerification(Class< | |
final Class<?> resultClass = result.toClass(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: Existing code. I am curious why we put an assertion here as this is an argument passed by our code. If we put assertion here, then the question is why not put it in other places as well. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree with that. It seems a bit overly defensive. I think it's there to verify that it's validating an output template rather than a state template. Obviously, in the place where this is being called, it should never have state. Maybe a better |
||
Type genericType = method.getGenericParameterTypes()[0]; | ||
genericType = resolveVariableWithClassContext(baseClass, genericType); | ||
if (!(genericType instanceof ParameterizedType)) { | ||
Optional<ParameterizedType> parameterized = getParameterizedType(genericType); | ||
if (!parameterized.isPresent()) { | ||
throw extractionError( | ||
"The method '%s' needs generic parameters for the CompletableFuture at position %d.", | ||
method.getName(), 0); | ||
} | ||
final Type returnType = ((ParameterizedType) genericType).getActualTypeArguments()[0]; | ||
// If verifyFutureContainsCollection is given, it is assumed to be a generic parameters | ||
// of argumentClass, also at the position genericPos | ||
final Type returnType; | ||
if (verifyFutureContainsCollection) { | ||
Type nestedGenericType = parameterized.get().getActualTypeArguments()[0]; | ||
Optional<ParameterizedType> nestedParameterized = | ||
getParameterizedType(nestedGenericType); | ||
if (!nestedParameterized.isPresent() | ||
|| !nestedParameterized.get().getRawType().equals(Collection.class)) { | ||
throw extractionError( | ||
"The method '%s' expects nested generic type CompletableFuture<Collection> for the %d arg.", | ||
method.getName(), 0); | ||
} | ||
returnType = nestedParameterized.get().getActualTypeArguments()[0]; | ||
} else { | ||
returnType = parameterized.get().getActualTypeArguments()[0]; | ||
} | ||
Class<?> returnTypeClass = getClassFromType(returnType); | ||
// Parameters should be validated using strict autoboxing. | ||
// For return types, we can be more flexible as the UDF should know what it declared. | ||
|
Uh oh!
There was an error while loading. Please reload this page.