diff --git a/docs/content.zh/docs/dev/table/functions/udfs.md b/docs/content.zh/docs/dev/table/functions/udfs.md index bd4771fba2dae..40e3ea29963bf 100644 --- a/docs/content.zh/docs/dev/table/functions/udfs.md +++ b/docs/content.zh/docs/dev/table/functions/udfs.md @@ -1234,28 +1234,28 @@ env.sqlQuery( {{< top >}} -Asynchronous Table Functions +异步表值函数 ---------------- -Similar to `AsyncScalarFunction`, there also exists a `AsyncTableFunction` for returning multiple row results rather than a single scalar value. Similarly, this is most useful when interacting with external systems (for example when enriching stream events with data stored in a database). +`AsyncTableFunction` 与 `AsyncScalarFunction` 类似,区别在于它返回的是多行结果而不是单个标量值。它最适合的场景同样是与外部系统打交道——例如访问数据库,做流上的维表打宽。 -Asynchronous interaction with an external system means that a single function instance can handle many requests concurrently and receive the responses concurrently. That way, the waiting time can be overlaid with sending other requests and receiving responses. At the very least, the waiting time is amortized over multiple requests. This leads in most cases to much higher streaming throughput. +异步访问外部系统意味着同一个函数实例可以并发地发出多个请求、并发地接收响应;等待时间因此可以与其他请求的发送、响应的接收相互重叠。可以让等待开销分摊到多次请求上。在大多数场景下,这都会显著提高流处理的吞吐量。 -#### Defining an AsyncTableFunction +#### 定义 AsyncTableFunction -A user-defined asynchronous table function maps zero, one, or multiple scalar values to zero, one, or multiple Rows. Any data type listed in the [data types section]({{< ref "docs/sql/reference/data-types" >}}) can be used as a parameter or return type of an evaluation method. +用户自定义的异步表值函数接受零个、一个或多个参数,每次输入会输出零行、一行或多行结果。[数据类型部分]({{< ref "docs/sql/reference/data-types" >}}) 中列出的任何数据类型都可以作为求值方法的参数或返回类型。 -In order to define an asynchronous table function, extend the base class `AsyncTableFunction` in `org.apache.flink.table.functions` and implement one or more evaluation methods named `eval(...)`. The first argument must be a `CompletableFuture<...>` which is used to return the result, with subsequent arguments being the parameters passed to the function. +要定义一个异步表值函数,需要继承 `org.apache.flink.table.functions` 中的基类 `AsyncTableFunction`,并实现一个或多个名为 `eval(...)` 的求值方法。`eval` 的第一个参数必须是 `CompletableFuture<...>`,用于把结果回写给框架;之后的参数即对外暴露的函数实参。 -The number of outstanding calls to `eval` may be configured by [`table.exec.async-table.max-concurrent-operations`]({{< ref "docs/dev/table/config#table-exec-async-table-max-concurrent-operations" >}}). +`eval` 的并发调用数量上限可以通过 [`table.exec.async-table.max-concurrent-operations`]({{< ref "docs/dev/table/config#table-exec-async-table-max-concurrent-operations" >}}) 进行配置。 -#### Asynchronous Semantics -While calls to an `AsyncTableFunction` may be completed out of the original input order, to maintain correct semantics, the outputs of the function are guaranteed to maintain that input order to downstream components of the query. The data itself could reveal completion order (e.g. by containing fetch timestamps), so the user should consider whether this is acceptable for their use-case. +#### 异步语义 +对 `AsyncTableFunction` 的并发调用,其完成顺序可能与输入顺序不一致;但为了保证语义正确,框架仍会按输入顺序把函数输出交给查询下游的算子。即便如此,数据本身仍可能反映实际的完成顺序(例如携带获取时间戳);这一点是否会影响业务,需要使用方自行判断。 -#### Error Handling -The primary way for a user to indicate an error is to call `completableFuture.completeExceptionally(throwable)`. Similarly, if an exception is encountered by the system when invoking `eval`, that will also result in an error. When an error occurs, the system will consider the retry strategy, configured by [`table.exec.async-table.retry-strategy`]({{< ref "docs/dev/table/config#table-exec-async-table-retry-strategy" >}}). If this is `NO_RETRY`, the job is failed. If it is set to `FIXED_DELAY`, a period of [`table.exec.async-table.retry-delay`]({{< ref "docs/dev/table/config#table-exec-async-table-retry-delay" >}}) will be waited, and the function call will be retried. If there have been [`table.exec.async-table.max-attempts`]({{< ref "docs/dev/table/config#table-exec-async-table-max-attempts" >}}) failed attempts or if the timeout [`table.exec.async-table.timeout`]({{< ref "docs/dev/table/config#table-exec-async-table-timeout" >}}) expires (including all retry attempts), the job will fail. +#### 错误处理 +上报错误的主要方式是调用 `CompletableFuture.completeExceptionally(Throwable)`;此外,如果框架在调用 `eval` 时本身就抛出了异常,同样会被视为出错。出错后系统会按照重试策略处理,该策略由 [`table.exec.async-table.retry-strategy`]({{< ref "docs/dev/table/config#table-exec-async-table-retry-strategy" >}}) 控制:若设为 `NO_RETRY`,作业直接失败;若设为 `FIXED_DELAY`,则会在等待 [`table.exec.async-table.retry-delay`]({{< ref "docs/dev/table/config#table-exec-async-table-retry-delay" >}}) 之后再次发起调用。当失败次数达到 [`table.exec.async-table.max-attempts`]({{< ref "docs/dev/table/config#table-exec-async-table-max-attempts" >}}) 上限,或总耗时(含全部重试)超过 [`table.exec.async-table.timeout`]({{< ref "docs/dev/table/config#table-exec-async-table-timeout" >}}),作业仍会失败。 -The following example shows how to do work on a thread pool in the background, though any libraries exposing an async interface may be directly used to complete the `CompletableFuture` from a callback. See the [Implementation Guide](#implementation-guide) for more details. +下面的示例演示了如何借助一个后台线程池完成异步任务;当然,任何暴露异步接口的库都可以直接拿来在回调中完成 `CompletableFuture`。更多细节请参阅[实现指南](#implementation-guide)。 ```java import org.apache.flink.table.api.*; @@ -1298,24 +1298,73 @@ TableEnvironment env = TableEnvironment.create(...); env.getConfig().set("table.exec.async-table.max-concurrent-operations", "5"); env.getConfig().set("table.exec.async-table.timeout", "1m"); -// call function "inline" without registration in Table API +// 在 Table API 中"内联"调用函数,无需注册 env.from("MyTable") .joinLateral(call(BackgroundFunction.class, $("myField"))) .select($("*")) -// register function +// 注册函数 env.createTemporarySystemFunction("BackgroundFunction", BackgroundFunction.class); -// call registered function in Table API +// 在 Table API 中调用已注册的函数 env.from("MyTable") .joinLateral(call("BackgroundFunction", $("myField"))) .select($("*")) -// call registered function in SQL +// 在 SQL 中调用已注册的函数 env.sqlQuery("SELECT * FROM MyTable, LATERAL TABLE(BackgroundFunction(myField))"); ``` +#### 自定义超时处理 + +默认情况下,如果某次 `eval` 调用在 [`table.exec.async-table.timeout`]({{< ref "docs/dev/table/config#table-exec-async-table-timeout" >}}) 之内仍未完成 `CompletableFuture`,且配置的重试也已用尽,框架会抛出 `java.util.concurrent.TimeoutException` 并让作业失败。如果希望覆盖这一行为,可以在 `AsyncTableFunction` 子类中声明一个与之匹配的 `timeout(...)` 方法,由它返回一行兜底数据,或由用户自定义超时处理。这通常在使用`AsyncTableFunction`发送请求调用远程LLM集群时很有用,众所周知,请求远程LLM集群出现超时的概率相对而言更高,相当多的用户希望超时不要直接导致作业失败,而是有自定义的兜底行为,`timeout`方法为此而生。 + +`timeout` 方法需要**同时**满足以下条件: + +- **方法声明。** 必须是 `public` 实例方法(不能是 `static`),方法名固定为 `timeout`。 +- **签名要与 `eval` 对齐。** 参数列表需要与对应的 `eval` 严格一致:第一个参数是 `CompletableFuture`,其泛型类型必须与 `eval` **完全相同**;其余参数的类型和顺序也必须与 `eval` **完全相同**。`timeout` 支持重载——每一个需要兜底逻辑的 `eval` 重载都可以单独声明一个对应的 `timeout`。 +- **同步完成(强制要求)。** 该回调会在算子的 mailbox 线程上运行,**必须**在方法返回前完成 future。方法返回后,框架会立即检查 `future.isDone()`;如果尚未完成,会强制以 `IllegalStateException` 把 future 完成。也就是说: + - **不要**在方法体内再发起一次异步调用(例如又一次重试或二次查询),并指望它的回调最终把 future 完成——等回调真正触发时,框架已经放弃了这条记录。 + - **不要**启动新线程异步去完成 future,理由同上。 + - 方法体只该做一件事:给出一个轻量的兜底结果,比如返回一个常量行、一个 NULL 行、空集合,或者用业务自定义的异常调用 `completeExceptionally(...)`。 +- **抛异常是安全的。** 直接在方法体内同步抛出异常没有问题:框架会把异常转交给下游的 `ResultFuture`,效果等价于显式调用 `future.completeExceptionally(thrown)`。 + +在以上约束之外,框架还会在以下场景保证稳定的行为: + +- **未声明 `timeout` 方法。** 沿用框架默认的 `TimeoutException`,不会引发代码生成阶段的失败。 +- **签名不兼容。** 如果 `timeout` 方法可见性合法,但参数列表无法和当前调用点的实参类型对应,框架会在规划期(代码生成阶段)以 `ValidationException` 直接失败;错误信息中会包含函数的全限定类名,并同时给出期望签名与实际签名。 +- **空集合兜底。** 调用 `future.complete(Collections.emptyList())` 时:对 INNER lookup join 而言该行会被丢弃;对 LEFT OUTER lookup join 而言,右表字段会被填充为 NULL。 + +下面这段示例在 `BackgroundFunction` 的基础上加了一个 `timeout` 回调,展示如何同步地完成 future 并返回一行兜底数据: + +```java +import org.apache.flink.table.functions.AsyncTableFunction; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; + +public static class BackgroundFunctionWithTimeout extends AsyncTableFunction { + + public void eval(CompletableFuture> future, Integer waitMax) { + // ...逻辑与上面的 BackgroundFunction 一致:把异步任务派发出去, + // 然后在回调线程中完成 `future`。 + } + + // 参数列表必须与 eval() 对齐:CompletableFuture 的泛型相同, + // 后续参数(Integer waitMax)的类型和顺序也相同。 + // 方法体必须同步完成 `future`——不要再发起异步调用, + // 也不要启动新线程异步去完成它;这个回调被触发时, + // 算子已经放弃了这条记录。 + public void timeout(CompletableFuture> future, Integer waitMax) { + future.complete(Collections.singletonList(-1L)); + // 也可以选择用业务自定义异常代替默认的 TimeoutException: + // future.completeExceptionally(new MyLookupTimeoutException(waitMax)); + } +} +``` + {{< top >}} 聚合函数 diff --git a/docs/content/docs/dev/table/functions/udfs.md b/docs/content/docs/dev/table/functions/udfs.md index 9441a28e1581b..405e0631e49ea 100644 --- a/docs/content/docs/dev/table/functions/udfs.md +++ b/docs/content/docs/dev/table/functions/udfs.md @@ -1358,6 +1358,56 @@ env.sqlQuery("SELECT * FROM MyTable, LATERAL TABLE(BackgroundFunction(myField))" ``` +#### Custom Timeout Handling + +By default, when an `eval` invocation fails to complete the `CompletableFuture` within [`table.exec.async-table.timeout`]({{< ref "docs/dev/table/config#table-exec-async-table-timeout" >}}) (after exhausting any configured retries), the framework surfaces a `java.util.concurrent.TimeoutException` and fails the job. To override this behavior, you can declare a matching `timeout(...)` method in your `AsyncTableFunction` subclass, which returns a fallback row or implements custom timeout handling logic. This is particularly useful when using `AsyncTableFunction` to send requests to a remote LLM cluster. As is well known, requests to remote LLM clusters are relatively more prone to timeouts, and many users prefer that a timeout does not directly cause the job to fail, but instead triggers custom fallback behavior. The timeout method is designed precisely for this purpose. + +The `timeout` method must satisfy **all** of the following: + +- **Declaration.** Public, non-static, named `timeout`. +- **Signature parity with `eval`.** The parameter list mirrors the matching `eval`: the first parameter is a `CompletableFuture` with the **same** generic type as in `eval`; the remaining parameters are the lookup keys with the **same** types and order. Overloads are supported — declare one `timeout` per `eval` overload that needs a fallback. +- **Synchronous completion (enforced).** The handler runs on the operator's mailbox thread and **must** complete the future before it returns. The framework checks `future.isDone()` immediately after the call and, if not, forces completion with `IllegalStateException`. Concretely: + - Do **not** issue another async call (e.g. a retry, a secondary lookup) and rely on its callback to complete the future — by the time the callback fires, the framework has already short-circuited this record. + - Do **not** spawn a thread that completes the future asynchronously, for the same reason. + - The body should be a pure, cheap fallback: a constant row, a NULL row, an empty collection, or `completeExceptionally(...)` with a user-defined exception. +- **Exception transparency.** Throwing synchronously from the body is safe: the throw is forwarded to the downstream `ResultFuture`, equivalent to calling `future.completeExceptionally(thrown)`. + +Additional behavior the framework guarantees on top of the convention: + +- **No `timeout` method** → the default `TimeoutException` fires; no codegen failure. +- **Incompatible signature** → if a `timeout` method exists with valid visibility but its parameter list is not assignable from the call site's lookup-key types, the framework fails fast during planning with a `ValidationException` whose message includes the function's fully qualified class name and the expected / actual signatures. +- **Empty-collection fallback** → `future.complete(Collections.emptyList())` drops the row for an INNER lookup join and pads the right side with NULL for a LEFT OUTER lookup join. + +The following example extends `BackgroundFunction` with a `timeout` handler that completes the future synchronously with a fallback row: + +```java +import org.apache.flink.table.functions.AsyncTableFunction; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; + +public static class BackgroundFunctionWithTimeout extends AsyncTableFunction { + + public void eval(CompletableFuture> future, Integer waitMax) { + // ...same as BackgroundFunction above — kicks off async work and + // completes `future` from the callback thread. + } + + // The parameter list MUST mirror eval(): same CompletableFuture generic + // type, and the same trailing argument list (Integer waitMax). + // The body MUST complete `future` synchronously — do NOT issue another + // async call or spawn a thread to complete it later, because the operator + // has already abandoned this record by the time this handler runs. + public void timeout(CompletableFuture> future, Integer waitMax) { + future.complete(Collections.singletonList(-1L)); + // Alternatively, surface a domain-specific error instead of the default + // TimeoutException: + // future.completeExceptionally(new MyLookupTimeoutException(waitMax)); + } +} +``` + {{< top >}} Aggregate Functions diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/AsyncTableFunction.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/AsyncTableFunction.java index 2b52cc16ed87a..6358bc1639499 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/AsyncTableFunction.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/AsyncTableFunction.java @@ -23,13 +23,13 @@ import org.apache.flink.table.annotation.FunctionHint; import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.connector.source.LookupTableSource; -import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.extraction.TypeInferenceExtractor; import org.apache.flink.table.types.inference.TypeInference; import org.apache.flink.types.Row; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; /** * Base class for a user-defined asynchronous table function. A user-defined asynchronous table @@ -72,32 +72,99 @@ * An error can also be propagated to the async IO operator by calling {@link * CompletableFuture#completeExceptionally(Throwable)}. * + *

Optionally, a custom timeout handler can be defined by convention: when an async invocation + * exceeds the configured timeout, the framework invokes the matching timeout method to + * let the function supply a fallback result (via {@link CompletableFuture#complete}) or surface a + * domain-specific error (via {@link CompletableFuture#completeExceptionally(Throwable)}) instead of + * the default {@link TimeoutException}. + * + *

A timeout method must satisfy all of the following constraints: + * + *

    + *
  • Declaration. Declared publicly, not static, and named timeout. + *
  • Signature parity with eval. The parameter list mirrors the matching + * eval: the first parameter is a {@link CompletableFuture} with the + * same generic type as in eval; the remaining parameters are the lookup + * keys with the same types and order. Overloads are supported — declare one + * timeout per eval overload that needs a fallback. + *
  • Synchronous completion (enforced). The handler must complete the future before + * it returns. It is invoked on the operator's mailbox thread, so blocking or scheduling + * additional async work there would stall the entire operator; the framework checks {@code + * future.isDone()} immediately after the call and, if it's not, forces the future to complete + * with an {@link IllegalStateException}. Concretely: + *
      + *
    • Do not issue another async call (e.g. a retry, a secondary lookup) and rely + * on its callback to complete the future — by the time the callback fires, the + * framework has already short-circuited this record with the IllegalStateException + * above. + *
    • Do not spawn a thread that completes the future asynchronously, for the same + * reason. + *
    • The body should be a pure, cheap fallback: a constant row, a NULL row, an empty + * collection, or {@code completeExceptionally(...)} with a user-defined exception. + *
    + *
  • Exception transparency. Throwing synchronously from the body is safe: the throw + * propagates up the framework's outer catch around the {@code timeout(...)} dispatch and is + * forwarded to the downstream {@code ResultFuture}, equivalent in effect to calling {@code + * future.completeExceptionally(thrown)}. You do not need to wrap the body in try/catch + * yourself. + *
+ * + *

Error and dispatch behavior the framework guarantees on top of the constraints above: + * + *

    + *
  • Absent handler → the default {@link TimeoutException} fires; no codegen failure. + *
  • Incompatible signature → if a timeout method exists with valid + * visibility but its parameter list is not assignable from the current call site's lookup-key + * types, the framework fails fast during planning (code generation) with a {@link + * org.apache.flink.table.api.ValidationException} whose message includes the function's fully + * qualified class name, the expected signature, and the actual signatures found. + *
  • Overload resolution → when multiple timeout overloads are declared, the + * one matching the current call site's lookup-key types is dispatched (decoy overloads with + * incompatible types or arity are tolerated, never invoked). + *
  • Empty-collection fallback → {@code future.complete(emptyList())} drops the row for + * an INNER lookup join and pads the right side with NULL for a LEFT OUTER lookup join. + *
+ * *

For storing a user-defined function in a catalog, the class must have a default constructor * and must be instantiable during runtime. Anonymous functions in Table API can only be persisted * if the function is not stateful (i.e. containing only transient and static fields). * - *

The following example shows how to perform an asynchronous request to Apache HBase: + *

The following example shows how to perform an asynchronous remote model invocation, with a + * fallback handler provided via the timeout convention: * *

{@code
- * public class HBaseAsyncTableFunction extends AsyncTableFunction {
+ * public class RemoteModelAsyncTableFunction extends AsyncTableFunction {
+ *
+ *   private transient RemoteModelClient client;
+ *
+ *   public void open(FunctionContext context) {
+ *     client = new RemoteModelClient(...);
+ *   }
  *
  *   // implement an "eval" method that takes a CompletableFuture as the first parameter
  *   // and ends with as many parameters as you want
- *   public void eval(CompletableFuture> result, String rowkey) {
- *     Get get = new Get(Bytes.toBytes(rowkey));
- *     ListenableFuture future = hbase.asyncGet(get);
- *     Futures.addCallback(future, new FutureCallback() {
- *       public void onSuccess(Result hbaseResult) {
- *         List ret = process(hbaseResult);
- *         result.complete(ret);
- *       }
- *       public void onFailure(Throwable thrown) {
- *         result.completeExceptionally(thrown);
+ *   public void eval(CompletableFuture> result, String prompt) {
+ *     CompletableFuture modelFuture = client.predictAsync(prompt);
+ *     modelFuture.whenComplete((response, throwable) -> {
+ *       if (throwable != null) {
+ *         result.completeExceptionally(throwable);
+ *       } else {
+ *         result.complete(
+ *             Collections.singletonList(GenericRowData.of(StringData.fromString(response))));
  *       }
  *     });
  *   }
  *
- *   // you can overload the eval method here ...
+ *   // implement a "timeout" method whose parameter list mirrors the matching "eval" method.
+ *   // The body MUST complete the future synchronously — do NOT call client.predictAsync(...)
+ *   // again or spawn a thread to complete the future, because the operator has already
+ *   // abandoned this record by the time this handler runs.
+ *   public void timeout(CompletableFuture> result, String prompt) {
+ *     result.complete(
+ *         Collections.singletonList(GenericRowData.of(StringData.fromString("FALLBACK"))));
+ *   }
+ *
+ *   // you can overload the eval/timeout methods here ...
  * }
  * }
* 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 4f06e3a5bf85c..d32c9dc28d3f4 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 @@ -97,6 +97,8 @@ public final class UserDefinedFunctionHelper { public static final String ASYNC_TABLE_EVAL = "eval"; + public static final String ASYNC_TABLE_TIMEOUT = "timeout"; + public static final String PROCESS_TABLE_EVAL = "eval"; public static final String PROCESS_TABLE_ON_TIMER = "onTimer"; @@ -351,6 +353,94 @@ public static void validateClassForRuntime( } } + /** + * Validates whether an {@link AsyncTableFunction} subclass declares a usable {@code timeout} + * fallback method that mirrors the {@code eval(CompletableFuture, ...)} signature. + * + *

The detection uses a three-step contract: collect every method literally named {@code + * timeout} via {@link ExtractionUtils#collectMethods}, filter to those that are both {@code + * public} and non-{@code static}, and finally verify the surviving candidates against the + * expected {@code (CompletableFuture>, argumentClasses)} signature. + * + *

    + *
  • No applicable candidate (none declared, or all private/static/mis-spelled) — returns + * {@code false}; the framework falls back to {@link + * java.util.concurrent.TimeoutException} via the default {@code AsyncFunction#timeout}. + *
  • One or more applicable candidates with a matching signature — returns {@code true}; + * codegen will emit a {@code fetcher.timeout(...)} dispatch. + *
  • One or more applicable candidates but signature mismatch — throws {@link + * ValidationException} eagerly with the FQN, the expected signature, and every actual + * candidate signature so users can locate the offending method quickly. + *
+ */ + public static boolean validateAsyncTableFunctionTimeoutClass( + Class functionClass, + Class[] argumentClasses, + String functionName) { + final List candidates = + ExtractionUtils.collectMethods(functionClass, ASYNC_TABLE_TIMEOUT); + final List applicable = + candidates.stream() + .filter( + method -> + Modifier.isPublic(method.getModifiers()) + && !Modifier.isStatic(method.getModifiers())) + .collect(Collectors.toList()); + if (applicable.isEmpty()) { + return false; + } + // Mirror the eval convention: prepend the implicit CompletableFuture parameter so the + // full expected signature is `timeout(CompletableFuture, argumentClasses...)`. + final Class[] expectedSignature = new Class[argumentClasses.length + 1]; + expectedSignature[0] = CompletableFuture.class; + System.arraycopy(argumentClasses, 0, expectedSignature, 1, argumentClasses.length); + try { + validateClassForRuntime( + functionClass, + ASYNC_TABLE_TIMEOUT, + expectedSignature, + void.class, + functionName); + } catch (ValidationException originalException) { + throw new ValidationException( + buildTimeoutSignatureMismatchMessage( + functionClass, expectedSignature, applicable), + originalException); + } + return true; + } + + private static String buildTimeoutSignatureMismatchMessage( + Class functionClass, + Class[] argumentClasses, + List applicable) { + final StringBuilder builder = new StringBuilder(); + builder.append("Could not find a public, non-static `timeout(") + .append(formatSignature(argumentClasses)) + .append(")` method in class ") + .append(functionClass.getName()) + .append(".\nExisting public timeout methods:"); + for (Method method : applicable) { + builder.append("\n - public "); + if (Modifier.isFinal(method.getModifiers())) { + builder.append("final "); + } + builder.append(method.getReturnType().getName()) + .append(' ') + .append(functionClass.getName()) + .append('.') + .append(method.getName()) + .append('(') + .append(formatSignature(method.getParameterTypes())) + .append(')'); + } + return builder.toString(); + } + + private static String formatSignature(Class[] argumentClasses) { + return Arrays.stream(argumentClasses).map(Class::getName).collect(Collectors.joining(", ")); + } + /** * Creates the runtime implementation of a {@link FunctionDefinition} as an instance of {@link * UserDefinedFunction}. 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 71cb771065a9a..aecbf52710748 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 @@ -22,6 +22,7 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogFunction; import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.data.RowData; import org.apache.flink.table.resource.ResourceUri; import org.apache.flink.util.Collector; @@ -43,6 +44,7 @@ import static org.apache.flink.table.functions.UserDefinedFunctionHelper.instantiateFunction; import static org.apache.flink.table.functions.UserDefinedFunctionHelper.isClassNameSerializable; import static org.apache.flink.table.functions.UserDefinedFunctionHelper.prepareInstance; +import static org.apache.flink.table.functions.UserDefinedFunctionHelper.validateAsyncTableFunctionTimeoutClass; import static org.apache.flink.table.functions.UserDefinedFunctionHelper.validateClass; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -101,6 +103,132 @@ void testValidation(TestSpec testSpec) { } } + // -------------------------------------------------------------------------------------------- + // Tests for validateAsyncTableFunctionTimeoutClass (T017 / T020) + // -------------------------------------------------------------------------------------------- + + /** US2 (T017): absent / private / static / misspelled timeout methods return false. */ + @Test + void testValidateAsyncTimeoutReturnsFalseWhenNoApplicableMethod() { + final Class[] argumentClasses = new Class[] {Integer.class}; + + assertThat( + validateAsyncTableFunctionTimeoutClass( + NoTimeoutAsyncTableFunction.class, argumentClasses, "f")) + .isFalse(); + assertThat( + validateAsyncTableFunctionTimeoutClass( + PrivateTimeoutAsyncTableFunction.class, argumentClasses, "f")) + .isFalse(); + assertThat( + validateAsyncTableFunctionTimeoutClass( + StaticTimeoutAsyncTableFunction.class, argumentClasses, "f")) + .isFalse(); + assertThat( + validateAsyncTableFunctionTimeoutClass( + MisspelledTimeoutAsyncTableFunction.class, argumentClasses, "f")) + .isFalse(); + } + + /** US3 (T020): matching signature returns true. */ + @Test + void testValidateAsyncTimeoutAcceptsMatchingSignature() { + final Class[] argumentClasses = new Class[] {Integer.class}; + assertThat( + validateAsyncTableFunctionTimeoutClass( + ValidTimeoutAsyncTableFunction.class, argumentClasses, "f")) + .isTrue(); + } + + /** US3 (T020): wrong arg count fails fast with FQN + expected + actual signature. */ + @Test + void testValidateAsyncTimeoutRejectsWrongArgCount() { + final Class[] argumentClasses = new Class[] {Integer.class, String.class}; + assertThatThrownBy( + () -> + validateAsyncTableFunctionTimeoutClass( + WrongArgCountTimeoutAsyncTableFunction.class, + argumentClasses, + "f")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining(WrongArgCountTimeoutAsyncTableFunction.class.getName()) + .hasMessageContaining(Integer.class.getName()) + .hasMessageContaining(String.class.getName()); + } + + /** US3 (T020): wrong arg type fails fast with FQN + expected + actual signature. */ + @Test + void testValidateAsyncTimeoutRejectsWrongArgType() { + final Class[] argumentClasses = new Class[] {Integer.class}; + assertThatThrownBy( + () -> + validateAsyncTableFunctionTimeoutClass( + WrongArgTypeTimeoutAsyncTableFunction.class, + argumentClasses, + "f")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining(WrongArgTypeTimeoutAsyncTableFunction.class.getName()) + .hasMessageContaining(Integer.class.getName()) + .hasMessageContaining(String.class.getName()); + } + + /** US3 (T020): first arg not CompletableFuture fails fast. */ + @Test + void testValidateAsyncTimeoutRejectsMissingFutureArg() { + final Class[] argumentClasses = new Class[] {Integer.class}; + assertThatThrownBy( + () -> + validateAsyncTableFunctionTimeoutClass( + NoFutureTimeoutAsyncTableFunction.class, + argumentClasses, + "f")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining(NoFutureTimeoutAsyncTableFunction.class.getName()); + } + + /** + * Multiple {@code timeout} overloads coexist — the one whose parameter list matches the call + * site's keys must be resolved as valid, without being shadowed by the other overload. + */ + @Test + void testValidateAsyncTimeoutResolvesAmongOverloads() { + assertThat( + validateAsyncTableFunctionTimeoutClass( + MultipleOverloadTimeoutAsyncTableFunction.class, + new Class[] {Integer.class}, + "f")) + .isTrue(); + } + + /** + * Multiple {@code timeout} overloads are declared but none matches the call site's keys — the + * error message must enumerate every applicable candidate so users can locate the offending + * declaration. Asserts both overload signatures appear in full form. + */ + @Test + void testValidateAsyncTimeoutListsAllCandidatesWhenNoOverloadMatches() { + final String futureFqn = CompletableFuture.class.getName(); + assertThatThrownBy( + () -> + validateAsyncTableFunctionTimeoutClass( + MultipleOverloadTimeoutAsyncTableFunction.class, + new Class[] {Long.class}, + "f")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining(MultipleOverloadTimeoutAsyncTableFunction.class.getName()) + .hasMessageContaining(Long.class.getName()) + .hasMessageContaining("Existing public timeout methods") + .hasMessageContaining("timeout(" + futureFqn + ", " + Integer.class.getName() + ")") + .hasMessageContaining( + "timeout(" + + futureFqn + + ", " + + Integer.class.getName() + + ", " + + String.class.getName() + + ")"); + } + @Test void testSerialization() { assertThat(isClassNameSerializable(new ValidTableFunction())).isTrue(); @@ -468,4 +596,79 @@ public String eval() { return state; } } + + // -------------------------------------------------------------------------------------------- + // AsyncTableFunction subclasses used by validateAsyncTableFunctionTimeoutClass tests + // -------------------------------------------------------------------------------------------- + + /** No timeout method declared. */ + public static class NoTimeoutAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, Integer key) {} + } + + /** Private timeout — must be ignored. */ + public static class PrivateTimeoutAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, Integer key) {} + + private void timeout(CompletableFuture> future, Integer key) {} + } + + /** Static timeout — must be ignored. */ + public static class StaticTimeoutAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, Integer key) {} + + public static void timeout(CompletableFuture> future, Integer key) {} + } + + /** Method name misspelled — must be ignored. */ + public static class MisspelledTimeoutAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, Integer key) {} + + public void timout(CompletableFuture> future, Integer key) {} + } + + /** Valid timeout signature matching argumentClasses. */ + public static class ValidTimeoutAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, Integer key) {} + + public void timeout(CompletableFuture> future, Integer key) {} + } + + /** Timeout exists but accepts wrong number of args. */ + public static class WrongArgCountTimeoutAsyncTableFunction extends AsyncTableFunction { + public void eval( + CompletableFuture> future, Integer key1, String key2) {} + + public void timeout(CompletableFuture> future, Integer key) {} + } + + /** Timeout exists but accepts wrong arg type. */ + public static class WrongArgTypeTimeoutAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, Integer key) {} + + public void timeout(CompletableFuture> future, String key) {} + } + + /** Timeout first arg is not CompletableFuture. */ + public static class NoFutureTimeoutAsyncTableFunction extends AsyncTableFunction { + public void eval(CompletableFuture> future, Integer key) {} + + public void timeout(Integer key) {} + } + + /** + * Two {@code eval} overloads with paired {@code timeout} overloads. Used to verify both the + * matched-overload path (returns true for either signature) and the unmatched-overload path + * (error message must list both candidates). + */ + public static class MultipleOverloadTimeoutAsyncTableFunction + extends AsyncTableFunction { + public void eval(CompletableFuture> future, Integer key) {} + + public void eval(CompletableFuture> future, Integer k1, String k2) {} + + public void timeout(CompletableFuture> future, Integer key) {} + + public void timeout(CompletableFuture> future, Integer k1, String k2) {} + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/codegen/AsyncCodeGenerator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/codegen/AsyncCodeGenerator.java index 04b8e5ad31a3e..5a4bbc1118d55 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/codegen/AsyncCodeGenerator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/codegen/AsyncCodeGenerator.java @@ -96,7 +96,8 @@ public static GeneratedFunction> generateFunctio JavaScalaConversionUtil.toScala(Optional.empty()), JavaScalaConversionUtil.toScala(Optional.empty()), CodeGenUtils.DEFAULT_COLLECTOR_TERM(), - CodeGenUtils.DEFAULT_CONTEXT_TERM()); + CodeGenUtils.DEFAULT_CONTEXT_TERM(), + JavaScalaConversionUtil.toScala(Optional.empty())); } @SuppressWarnings("unchecked") 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 index de4647236370c..e21bad49ef7de 100644 --- 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 @@ -22,16 +22,24 @@ 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.codegen.calls.BridgingFunctionGenUtil; 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.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexVisitorImpl; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; +import scala.Option; +import scala.Tuple3; + /** * Generates an {@link AsyncFunction} which can be used to evaluate correlate invocations from an * async table function. @@ -46,20 +54,22 @@ public static GeneratedFunction> generateFunction ReadableConfig tableConfig, ClassLoader classLoader) { CodeGeneratorContext ctx = new CodeGeneratorContext(tableConfig, classLoader); - String processCode = - generateProcessCode(ctx, inputType, invocation, CodeGenUtils.DEFAULT_INPUT1_TERM()); + ProcessCode processCode = + generateProcessCode( + ctx, inputType, returnType, invocation, CodeGenUtils.DEFAULT_INPUT1_TERM()); return FunctionCodeGenerator.generateFunction( ctx, name, getFunctionClass(), - processCode, + processCode.bodyCode, returnType, inputType, CodeGenUtils.DEFAULT_INPUT1_TERM(), JavaScalaConversionUtil.toScala(Optional.empty()), JavaScalaConversionUtil.toScala(Optional.empty()), CodeGenUtils.DEFAULT_COLLECTOR_TERM(), - CodeGenUtils.DEFAULT_CONTEXT_TERM()); + CodeGenUtils.DEFAULT_CONTEXT_TERM(), + JavaScalaConversionUtil.toScala(processCode.timeoutBodyCode)); } @SuppressWarnings("unchecked") @@ -67,8 +77,12 @@ private static Class> getFunctionClass() { return (Class>) (Object) AsyncFunction.class; } - private static String generateProcessCode( - CodeGeneratorContext ctx, RowType inputType, RexCall invocation, String inputTerm) { + private static ProcessCode generateProcessCode( + CodeGeneratorContext ctx, + RowType inputType, + RowType returnType, + RexCall invocation, + String inputTerm) { invocation.accept(new AsyncCorrelateFunctionsValidator()); ExprCodeGenerator exprGenerator = @@ -78,8 +92,39 @@ private static String generateProcessCode( inputTerm, JavaScalaConversionUtil.toScala(Optional.empty())); - GeneratedExpression invocationExprs = exprGenerator.generateExpression(invocation); - return invocationExprs.code(); + // Generate operand expressions explicitly so we can call the timeout-aware helper below. + // Letting `generateExpression(invocation)` walk the call instead would route through + // BridgingSqlFunctionCallGen, which discards the optional `timeout(...)` call we need to + // render into the generated AsyncFunction subclass. + List operands = new ArrayList<>(); + for (RexNode operand : invocation.getOperands()) { + operands.add(exprGenerator.generateExpression(operand)); + } + + Tuple3, DataType> result = + BridgingFunctionGenUtil.generateBridgingFunctionCallWithTimeout( + ctx, + invocation, + null, + JavaScalaConversionUtil.toScala(operands), + returnType, + false); + + Option timeoutCall = result._2(); + Optional timeoutBodyCode = + timeoutCall.isDefined() ? Optional.of(timeoutCall.get().code()) : Optional.empty(); + return new ProcessCode(result._1().code(), timeoutBodyCode); + } + + /** Pair of generated bodies for the eval (asyncInvoke) and optional timeout methods. */ + private static final class ProcessCode { + final String bodyCode; + final Optional timeoutBodyCode; + + ProcessCode(String bodyCode, Optional timeoutBodyCode) { + this.bodyCode = bodyCode; + this.timeoutBodyCode = timeoutBodyCode; + } } private static class AsyncCorrelateFunctionsValidator extends RexVisitorImpl { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCallCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCallCodeGenerator.scala index 438f14c21e38b..f92329e7d4c80 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCallCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCallCodeGenerator.scala @@ -76,6 +76,12 @@ object FunctionCallCodeGenerator { |""".stripMargin } + val adapted + : (CodeGeneratorContext, FunctionCallContext, UserDefinedFunction, Seq[GeneratedExpression]) => (GeneratedExpression, Option[GeneratedExpression], DataType) = + (ctx, callContext, udf, operands) => { + val (call, dataType) = inferCall(ctx, callContext, udf, operands) + (call, None, dataType) + } generateFunctionCall( classOf[FlatMapFunction[RowData, RowData]], tableConfig, @@ -86,7 +92,7 @@ object FunctionCallCodeGenerator { collectorOutputType, parameters, syncFunctionDefinition, - inferCall, + adapted, functionName, generateClassName, fieldCopy, @@ -111,6 +117,51 @@ object FunctionCallCodeGenerator { Seq[GeneratedExpression]) => (GeneratedExpression, DataType), functionName: String, generateClassName: String + ): GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = { + val adapted + : (CodeGeneratorContext, FunctionCallContext, UserDefinedFunction, Seq[GeneratedExpression]) => (GeneratedExpression, Option[GeneratedExpression], DataType) = + (ctx, callContext, udf, operands) => { + val (call, dataType) = generateCallWithDataType(ctx, callContext, udf, operands) + (call, None, dataType) + } + generateAsyncFunctionTimeoutCall( + tableConfig, + classLoader, + dataTypeFactory, + inputType, + functionOutputType, + collectorOutputType, + parameters, + asyncFunctionDefinition, + adapted, + functionName, + generateClassName + ) + } + + /** + * Generates an async function ([[AsyncTableFunction]]) call that may additionally render a + * `timeout(...)` method body in the generated `RichAsyncFunction` subclass. The third element of + * the callback tuple, when present, is the timeout call expression produced by + * [[org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil .generateFunctionAwareCallWithDataTypeAndTimeout]]. + * When absent, the generated class inherits AsyncFunction's default timeout behaviour. + */ + def generateAsyncFunctionTimeoutCall( + tableConfig: ReadableConfig, + classLoader: ClassLoader, + dataTypeFactory: DataTypeFactory, + inputType: LogicalType, + functionOutputType: LogicalType, + collectorOutputType: LogicalType, + parameters: util.List[FunctionParam], + asyncFunctionDefinition: AsyncTableFunction[_], + generateCallWithTimeoutAndDataType: ( + CodeGeneratorContext, + FunctionCallContext, + UserDefinedFunction, + Seq[GeneratedExpression]) => (GeneratedExpression, Option[GeneratedExpression], DataType), + functionName: String, + generateClassName: String ): GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = { generateFunctionCall( classOf[AsyncFunction[RowData, AnyRef]], @@ -122,7 +173,7 @@ object FunctionCallCodeGenerator { collectorOutputType, parameters, asyncFunctionDefinition, - generateCallWithDataType, + generateCallWithTimeoutAndDataType, functionName, generateClassName, fieldCopy = true, @@ -140,11 +191,11 @@ object FunctionCallCodeGenerator { collectorOutputType: LogicalType, parameters: util.List[FunctionParam], functionDefinition: UserDefinedFunction, - generateCallWithDataType: ( + generateCallWithTimeoutAndDataType: ( CodeGeneratorContext, FunctionCallContext, UserDefinedFunction, - Seq[GeneratedExpression]) => (GeneratedExpression, DataType), + Seq[GeneratedExpression]) => (GeneratedExpression, Option[GeneratedExpression], DataType), functionName: String, generateClassName: String, fieldCopy: Boolean, @@ -172,18 +223,20 @@ object FunctionCallCodeGenerator { val ctx = new CodeGeneratorContext(tableConfig, classLoader) val operands = prepareOperands(ctx, inputType, parameters, fieldCopy) - val callWithDataType: (GeneratedExpression, DataType) = - generateCallWithDataType(ctx, callContext, udf, operands) + val (call, timeoutCall, dataType) = + generateCallWithTimeoutAndDataType(ctx, callContext, udf, operands) val function = FunctionCodeGenerator.generateFunction( ctx, generateClassName, generatedClass, - bodyCode(callWithDataType._1), + bodyCode(call), collectorOutputType, - inputType) + inputType, + timeoutBodyCode = timeoutCall.map(bodyCode) + ) - GeneratedTableFunctionWithDataType(function, callWithDataType._2) + GeneratedTableFunctionWithDataType(function, dataType) } private def prepareOperands( diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCodeGenerator.scala index 3131de4f8f909..e92e4e2627f6d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCodeGenerator.scala @@ -75,7 +75,8 @@ object FunctionCodeGenerator { input2Type: Option[LogicalType] = None, input2Term: Option[String] = Some(DEFAULT_INPUT2_TERM), collectorTerm: String = DEFAULT_COLLECTOR_TERM, - contextTerm: String = DEFAULT_CONTEXT_TERM): GeneratedFunction[F] = { + contextTerm: String = DEFAULT_CONTEXT_TERM, + timeoutBodyCode: Option[String] = None): GeneratedFunction[F] = { val funcName = newName(ctx, name) val inputTypeTerm = boxedTypeTermForType(input1Type) @@ -138,6 +139,27 @@ object FunctionCodeGenerator { throw new CodeGenException("Unsupported Function.") } + // Render an optional timeout(...) method for AsyncFunction subclasses. The body is supplied + // by the planner when the user UDF declares a legal `timeout` method (see + // BridgingFunctionGenUtil.generateAsyncTableFunctionTimeoutCall). When absent, the generated + // class falls back to AsyncFunction's default timeout that completes the future with a + // TimeoutException. + val timeoutMethodCode = timeoutBodyCode match { + case Some(body) if clazz == classOf[AsyncFunction[_, _]] => + s""" + |@Override + |public void timeout(Object _in1, + | org.apache.flink.streaming.api.functions.async.ResultFuture $collectorTerm) + | throws Exception { + | $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; + | ${ctx.reuseLocalVariableCode()} + | ${ctx.reuseInputUnboxingCode()} + | $body + |} + |""".stripMargin + case _ => "" + } + val funcCode = j""" ${ctx.getClassHeaderComment} @@ -166,6 +188,8 @@ object FunctionCodeGenerator { $bodyCode } + $timeoutMethodCode + @Override public void close() throws Exception { ${ctx.reuseCloseCode()} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala index e011dd2cfcec6..a2ad0122936ab 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala @@ -100,7 +100,7 @@ object LookupJoinCodeGenerator { lookupKeys: util.List[FunctionParam], asyncLookupFunction: AsyncTableFunction[_], functionName: String): GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = { - FunctionCallCodeGenerator.generateAsyncFunctionCall( + FunctionCallCodeGenerator.generateAsyncFunctionTimeoutCall( tableConfig, classLoader, dataTypeFactory, @@ -109,7 +109,7 @@ object LookupJoinCodeGenerator { returnType, lookupKeys, asyncLookupFunction, - generateCallWithDataType( + generateCallWithTimeoutAndDataType( dataTypeFactory, functionName, tableSourceType, @@ -170,6 +170,50 @@ object LookupJoinCodeGenerator { } } + private def generateCallWithTimeoutAndDataType( + dataTypeFactory: DataTypeFactory, + functionName: String, + tableSourceType: LogicalType, + baseClass: Class[_] + ) = ( + ctx: CodeGeneratorContext, + callContext: FunctionCallContext, + udf: UserDefinedFunction, + operands: Seq[GeneratedExpression]) => { + def inferCallWithTimeoutAndDataType( + ctx: CodeGeneratorContext, + callContext: FunctionCallContext, + udf: UserDefinedFunction, + operands: Seq[GeneratedExpression], + legacy: Boolean, + e: Exception = null): (GeneratedExpression, Option[GeneratedExpression], DataType) = { + val inference = createLookupTypeInference( + dataTypeFactory, + callContext, + baseClass, + udf, + functionName, + legacy, + e) + BridgingFunctionGenUtil.generateFunctionAwareCallWithDataTypeAndTimeout( + ctx, + operands, + tableSourceType, + inference, + callContext, + udf, + functionName, + skipIfArgsNull = true) + } + + try { + inferCallWithTimeoutAndDataType(ctx, callContext, udf, operands, legacy = false) + } catch { + case e: Exception => + inferCallWithTimeoutAndDataType(ctx, callContext, udf, operands, legacy = true, e) + } + } + /** * The [[LogicalType]] for inputs and output are known in a [[LookupTableSource]]. Thus, the * function's type inference is actually only necessary for enriching the conversion 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 2115e76304d96..3e0671752a9ef 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 @@ -33,6 +33,9 @@ import org.apache.flink.table.planner.codegen._ import org.apache.flink.table.planner.codegen.AsyncCodeGenerator.DEFAULT_DELEGATING_FUTURE_TERM import org.apache.flink.table.planner.codegen.CodeGenUtils._ import org.apache.flink.table.planner.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE} +import org.apache.flink.table.planner.delegation.PlannerBase +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction +import org.apache.flink.table.planner.functions.inference.OperatorBindingCallContext 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 @@ -49,6 +52,9 @@ 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 org.apache.calcite.rex.{RexCall, RexCallBinding, RexProgram} + +import java.util.Collections import java.util.concurrent.CompletableFuture import scala.collection.JavaConverters._ @@ -97,6 +103,93 @@ object BridgingFunctionGenUtil { udf: UserDefinedFunction, functionName: String, skipIfArgsNull: Boolean): (GeneratedExpression, DataType) = { + val result = generateFunctionAwareCallWithDataTypeAndTimeout( + ctx, + operands, + returnType, + inference, + callContext, + udf, + functionName, + skipIfArgsNull) + (result._1, result._3) + } + + /** + * Resolves the [[BridgingSqlFunction]] wrapping a [[RexCall]] to a [[UserDefinedFunction]] and + * generates both the eval-call expression and (for async table functions) an optional timeout- + * call expression. Mirrors [[BridgingSqlFunctionCallGen.generate]] but additionally returns the + * timeout-call so that callers driving an [[AsyncFunction]] codegen path (currently + * [[org.apache.flink.table.planner.codegen.AsyncCorrelateCodeGenerator]]) can render a custom + * `timeout(...)` method on the generated subclass. + * + *

Used in correlate-query codegen where the top-level [[RexCall]] is known to be an async + * table function invocation; lookup-join codegen still uses + * [[generateFunctionAwareCallWithDataTypeAndTimeout]] directly because it constructs the type + * inference from the [[org.apache.flink.table.connector.source.LookupTableSource]] schema rather + * than the function definition. + */ + def generateBridgingFunctionCallWithTimeout( + ctx: CodeGeneratorContext, + call: RexCall, + rexProgram: RexProgram, + operands: Seq[GeneratedExpression], + returnType: LogicalType, + skipIfArgsNull: Boolean): (GeneratedExpression, Option[GeneratedExpression], DataType) = { + val function = call.getOperator.asInstanceOf[BridgingSqlFunction] + val definition = function.getDefinition + val dataTypeFactory = function.getDataTypeFactory + val rexFactory = function.getRexFactory + + val callContext = new OperatorBindingCallContext( + dataTypeFactory, + definition, + RexCallBinding.create(function.getTypeFactory, call, rexProgram, Collections.emptyList()), + call.getType) + + val udf = UserDefinedFunctionHelper.createSpecializedFunction( + function.getName, + definition, + callContext, + classOf[PlannerBase].getClassLoader, + ctx.tableConfig, + new DefaultExpressionEvaluatorFactory(ctx.tableConfig, ctx.classLoader, rexFactory) + ) + val inference = udf.getTypeInference(dataTypeFactory) + + generateFunctionAwareCallWithDataTypeAndTimeout( + ctx, + operands, + returnType, + inference, + callContext, + udf, + function.toString, + skipIfArgsNull) + } + + /** + * Like [[generateFunctionAwareCallWithDataType]] but additionally returns an optional + * timeout-call expression when the UDF is an [[AsyncTableFunction]] (or its + * [[org.apache.flink.table.functions.AsyncLookupFunction]] subclass) that declares a public, + * non-static `timeout(CompletableFuture, ...)` method whose parameter list matches the call site. + * When the user UDF does not declare such a method, the returned timeout-call is empty and the + * generated `AsyncFunction` falls back to the framework default that completes the future with a + * [[java.util.concurrent.TimeoutException]]. + * + *

An illegal `timeout` signature (e.g., parameter count or types incompatible with the lookup + * keys) triggers a [[org.apache.flink.table.api.ValidationException]] that bubbles up to fail the + * job at submit / operator init time, so misconfigurations never reach the data path. + */ + def generateFunctionAwareCallWithDataTypeAndTimeout( + ctx: CodeGeneratorContext, + operands: Seq[GeneratedExpression], + returnType: LogicalType, + inference: TypeInference, + callContext: CallContext, + udf: UserDefinedFunction, + functionName: String, + skipIfArgsNull: Boolean): (GeneratedExpression, Option[GeneratedExpression], DataType) = { // enrich argument types with conversion class val castCallContext = TypeInferenceUtil.castArguments(inference, callContext, null) @@ -114,33 +207,45 @@ object BridgingFunctionGenUtil { enrichedOutputDataType, udf, functionName) - val call = generateFunctionAwareCall( + + val functionTerm = ctx.addReusableFunction(udf) + val externalOperands = prepareExternalOperands(ctx, operands, enrichedArgumentDataTypes) + + val call = generateFunctionAwareCallFromPreparedOperands( ctx, - operands, - enrichedArgumentDataTypes, + functionTerm, + externalOperands, enrichedOutputDataType, returnType, udf, skipIfArgsNull, None) - (call, enrichedOutputDataType) + + val timeoutCall = if (udf.getKind == FunctionKind.ASYNC_TABLE) { + generateAsyncTableFunctionTimeoutCall( + udf, + functionName, + enrichedArgumentDataTypes, + functionTerm, + externalOperands, + returnType, + enrichedOutputDataType, + skipIfArgsNull) + } else { + None + } + (call, timeoutCall, enrichedOutputDataType) } - private def generateFunctionAwareCall( + private def generateFunctionAwareCallFromPreparedOperands( ctx: CodeGeneratorContext, - operands: Seq[GeneratedExpression], - argumentDataTypes: Seq[DataType], + functionTerm: String, + externalOperands: Seq[GeneratedExpression], outputDataType: DataType, returnType: LogicalType, udf: UserDefinedFunction, skipIfArgsNull: Boolean, contextTerm: Option[String]): GeneratedExpression = { - - val functionTerm = ctx.addReusableFunction(udf) - - // operand conversion - val externalOperands = prepareExternalOperands(ctx, operands, argumentDataTypes) - if (udf.getKind == FunctionKind.TABLE || udf.getKind == FunctionKind.PROCESS_TABLE) { generateTableFunctionCall( ctx, @@ -170,6 +275,76 @@ object BridgingFunctionGenUtil { } } + /** + * Generates the body of the `timeout(...)` method that is rendered into a codegen + * `RichAsyncFunction` subclass when the user UDF declares a legal `timeout` method. Mirrors + * [[generateAsyncTableFunctionCall]] except the generated code invokes `function.timeout(...)` + * instead of `function.eval(...)` and reuses the SAME UDF instance registered by the eval path + * (see contracts/codegen-fetcher-timeout.md §2.1). + */ + private def generateAsyncTableFunctionTimeoutCall( + udf: UserDefinedFunction, + functionName: String, + enrichedArgumentDataTypes: Seq[DataType], + functionTerm: String, + externalOperands: Seq[GeneratedExpression], + returnType: LogicalType, + outputDataType: DataType, + skipIfArgsNull: Boolean): Option[GeneratedExpression] = { + val argumentClasses = enrichedArgumentDataTypes.map(_.getConversionClass).toArray + val hasTimeout = + validateAsyncTableFunctionTimeoutClass(udf.getClass, argumentClasses, functionName) + if (!hasTimeout) { + None + } else { + val DELEGATE_ASYNC_TABLE = className[DelegatingAsyncTableResultFuture] + val outputType = outputDataType.getLogicalType + val needsWrapping = !isCompositeType(outputType) + val isInternal = DataTypeUtils.isInternal(outputDataType) + // Enforce AsyncTableFunction's synchronous-completion contract for timeout(...): + // the handler MUST complete the future before it returns. If it doesn't (typically + // because the user issued another async call and stored the future for a later + // callback), the AsyncWaitOperator's ResultHandler would never be released and the + // downstream record would hang until shutdown. Fail fast here instead. + // + // We deliberately do NOT wrap the call in try/catch. Synchronous throws are already + // caught by AsyncCorrelateRunner.timeout / AsyncLookupJoinRunner.timeout and forwarded + // to the ResultFuture, so duplicating that here would only obscure the contract. + val callTimeoutCode = + s""" + |$functionTerm.timeout( + | delegates.getCompletableFuture(), + | ${externalOperands.map(_.resultTerm).mkString(", ")}); + |if (!delegates.getCompletableFuture().isDone()) { + | delegates.getCompletableFuture().completeExceptionally( + | new IllegalStateException( + | "AsyncTableFunction.timeout(...) must complete the future synchronously; " + | + "issuing another async call from inside timeout() is not allowed.")); + |} + |""".stripMargin + val functionCallCode = if (skipIfArgsNull) { + s""" + |${externalOperands.map(_.code).mkString("\n")} + |if (${externalOperands.map(_.nullTerm).mkString(" || ")}) { + | $DEFAULT_COLLECTOR_TERM.complete(java.util.Collections.emptyList()); + |} else { + | $DELEGATE_ASYNC_TABLE delegates = new $DELEGATE_ASYNC_TABLE($DEFAULT_COLLECTOR_TERM, + | $needsWrapping, $isInternal); + | $callTimeoutCode + |} + |""".stripMargin + } else { + s""" + |${externalOperands.map(_.code).mkString("\n")} + |$DELEGATE_ASYNC_TABLE delegates = new $DELEGATE_ASYNC_TABLE($DEFAULT_COLLECTOR_TERM, + | $needsWrapping, $isInternal); + |$callTimeoutCode + |""".stripMargin + } + Some(GeneratedExpression(NO_CODE, NEVER_NULL, functionCallCode, returnType)) + } + } + def generateTableFunctionCall( ctx: CodeGeneratorContext, functionTerm: String, diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TimeoutAsyncLookupTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TimeoutAsyncLookupTableFactory.java new file mode 100644 index 0000000000000..f2dff1cb64c9b --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TimeoutAsyncLookupTableFactory.java @@ -0,0 +1,914 @@ +/* + * 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.factories; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.LookupTableSource; +import org.apache.flink.table.connector.source.lookup.AsyncLookupFunctionProvider; +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.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.functions.AsyncLookupFunction; +import org.apache.flink.table.functions.AsyncTableFunction; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.legacy.connector.source.AsyncTableFunctionProvider; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * A test-only connector that exposes a configurable-delay async lookup, used to reproduce the + * timeout behaviour of async lookup join (driven by the {@code AsyncWaitOperator} via {@code + * AsyncLookupJoinRunner}). + * + *

Two provider modes are supported via the {@code provider-type} option: + * + *

    + *
  • {@code async-lookup-function} (default) — uses {@link AsyncLookupFunctionProvider} with an + * {@link AsyncLookupFunction} subclass. + *
  • {@code async-table-function} — uses the legacy {@link AsyncTableFunctionProvider} with an + * {@link AsyncTableFunction} that exposes an {@code eval(future, keys...)} method. + *
+ * + *

Each lookup sleeps for {@code lookup-delay} before completing. By comparing this delay against + * {@code table.exec.async-lookup.timeout} (or a per-join {@code LOOKUP('timeout'=...)} hint), a + * test can deterministically trigger or avoid an async timeout. + * + *

Both function variants additionally expose a non-standard {@code timeout(...)} method that + * completes the result future with {@link CustomLookupTimeoutException}. {@code + * FunctionCodeGenerator.scala} appends a {@code timeout(input, ResultFuture)} override to the + * generated {@code AsyncFunction} subclass that mirrors {@code asyncInvoke} with {@code eval} + * rewritten to {@code timeout}, so the user-side {@code timeout(future, keys...)} declared below is + * the one that ultimately runs when {@code AsyncWaitOperator} fires the timeout timer. + */ +public class TimeoutAsyncLookupTableFactory implements DynamicTableSourceFactory { + + public static final String IDENTIFIER = "timeout-async-lookup"; + + /** Selects which {@link LookupTableSource.LookupRuntimeProvider} the source returns. */ + public enum ProviderType { + ASYNC_LOOKUP_FUNCTION, + ASYNC_TABLE_FUNCTION, + /** + * Same as {@link #ASYNC_TABLE_FUNCTION} but the underlying UDF deliberately does NOT + * declare a {@code timeout(...)} method. Used to verify that the framework still works + * correctly (falls back to the default {@link java.util.concurrent.TimeoutException}) when + * the user supplies no custom timeout handler. + */ + ASYNC_TABLE_FUNCTION_WITHOUT_TIMEOUT, + /** + * Same as {@link #ASYNC_TABLE_FUNCTION} but the underlying UDF declares a {@code + * timeout(...)} method whose parameter signature is incompatible with the lookup keys. Used + * to verify that codegen fails fast with a {@link + * org.apache.flink.table.api.ValidationException ValidationException} carrying the UDF's + * FQN plus the expected and actual signatures. + */ + ASYNC_TABLE_FUNCTION_WITH_INCOMPATIBLE_TIMEOUT, + /** + * Same as {@link #ASYNC_TABLE_FUNCTION} but the {@code timeout(...)} method completes the + * future with a default row instead of an exception. The non-key columns are filled with + * the literal {@code "DEFAULT"}. Used to verify that the framework correctly forwards a + * synchronously-completed fallback result down the join pipeline so the job does not fail + * and the timed-out records carry the default value. + */ + ASYNC_TABLE_FUNCTION_WITH_DEFAULT_RESULT_TIMEOUT, + /** + * Same as {@link #ASYNC_TABLE_FUNCTION} but the {@code timeout(...)} method completes the + * future with an empty collection. Used to verify that a LEFT OUTER lookup join correctly + * pads the right side with NULL for every timed-out record (OUTER semantics must survive + * the timeout path), rather than dropping the left rows or failing the job. + */ + ASYNC_TABLE_FUNCTION_WITH_EMPTY_RESULT_TIMEOUT, + /** + * Same as {@link #ASYNC_TABLE_FUNCTION} but the UDF declares MULTIPLE {@code timeout(...)} + * overloads — only one matches the lookup-key signature. Used to verify that codegen routes + * to the matching overload (and that mere presence of extra overloads does not trip + * validation), via Java's compile-time overload resolution on the generated {@code + * functionTerm.timeout(...)} call site. + */ + ASYNC_TABLE_FUNCTION_WITH_OVERLOADED_TIMEOUT, + /** + * Same as {@link #ASYNC_TABLE_FUNCTION} but the {@code timeout(...)} method returns without + * completing the future (neither {@code complete} nor {@code completeExceptionally}). Used + * to verify that the codegen-emitted {@code isDone()} check fires fast with an {@link + * IllegalStateException} on the lookup-join path — symmetric coverage to the correlate-path + * test {@code testTimeoutNotCompletingFutureFailsFast}. + */ + ASYNC_TABLE_FUNCTION_WITH_NO_COMPLETION_TIMEOUT + } + + public static final ConfigOption LOOKUP_DELAY = + ConfigOptions.key("lookup-delay") + .durationType() + .defaultValue(Duration.ZERO) + .withDescription( + "How long each async lookup blocks before completing. Set this larger " + + "than table.exec.async-lookup.timeout to trigger a timeout."); + + public static final ConfigOption RETURNS_DATA = + ConfigOptions.key("returns-data") + .booleanType() + .defaultValue(false) + .withDescription( + "If true, the lookup returns a single row echoing the lookup key in " + + "the corresponding columns and leaving the rest as nulls. " + + "If false, the lookup returns an empty collection."); + + public static final ConfigOption PROVIDER_TYPE = + ConfigOptions.key("provider-type") + .enumType(ProviderType.class) + .defaultValue(ProviderType.ASYNC_LOOKUP_FUNCTION) + .withDescription( + "Which provider implementation to use: ASYNC_LOOKUP_FUNCTION (new " + + "AsyncLookupFunctionProvider, default) or ASYNC_TABLE_FUNCTION " + + "(legacy AsyncTableFunctionProvider)."); + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(LOOKUP_DELAY); + options.add(RETURNS_DATA); + options.add(PROVIDER_TYPE); + return options; + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + helper.validate(); + ReadableConfig options = helper.getOptions(); + return new TimeoutAsyncLookupTableSource( + options.get(LOOKUP_DELAY).toMillis(), + options.get(RETURNS_DATA), + options.get(PROVIDER_TYPE), + context.getPhysicalRowDataType().getLogicalType().getChildren().size()); + } + + private static class TimeoutAsyncLookupTableSource implements LookupTableSource { + + private final long delayMillis; + private final boolean returnsData; + private final ProviderType providerType; + private final int physicalFieldCount; + + TimeoutAsyncLookupTableSource( + long delayMillis, + boolean returnsData, + ProviderType providerType, + int physicalFieldCount) { + this.delayMillis = delayMillis; + this.returnsData = returnsData; + this.providerType = providerType; + this.physicalFieldCount = physicalFieldCount; + } + + @Override + public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { + int[] lookupKeyIndexes = Arrays.stream(context.getKeys()).mapToInt(k -> k[0]).toArray(); + switch (providerType) { + case ASYNC_LOOKUP_FUNCTION: + return AsyncLookupFunctionProvider.of( + new TimeoutAsyncLookupFunction( + delayMillis, + returnsData, + lookupKeyIndexes, + physicalFieldCount)); + case ASYNC_TABLE_FUNCTION: + return AsyncTableFunctionProvider.of( + new TimeoutAsyncTableFunction( + delayMillis, + returnsData, + lookupKeyIndexes, + physicalFieldCount)); + case ASYNC_TABLE_FUNCTION_WITHOUT_TIMEOUT: + return AsyncTableFunctionProvider.of( + new NoTimeoutAsyncTableFunction( + delayMillis, + returnsData, + lookupKeyIndexes, + physicalFieldCount)); + case ASYNC_TABLE_FUNCTION_WITH_INCOMPATIBLE_TIMEOUT: + return AsyncTableFunctionProvider.of( + new IncompatibleTimeoutAsyncTableFunction( + delayMillis, + returnsData, + lookupKeyIndexes, + physicalFieldCount)); + case ASYNC_TABLE_FUNCTION_WITH_DEFAULT_RESULT_TIMEOUT: + return AsyncTableFunctionProvider.of( + new DefaultResultTimeoutAsyncTableFunction( + delayMillis, + returnsData, + lookupKeyIndexes, + physicalFieldCount)); + case ASYNC_TABLE_FUNCTION_WITH_EMPTY_RESULT_TIMEOUT: + return AsyncTableFunctionProvider.of( + new EmptyResultTimeoutAsyncTableFunction( + delayMillis, + returnsData, + lookupKeyIndexes, + physicalFieldCount)); + case ASYNC_TABLE_FUNCTION_WITH_OVERLOADED_TIMEOUT: + return AsyncTableFunctionProvider.of( + new OverloadedTimeoutAsyncTableFunction( + delayMillis, + returnsData, + lookupKeyIndexes, + physicalFieldCount)); + case ASYNC_TABLE_FUNCTION_WITH_NO_COMPLETION_TIMEOUT: + return AsyncTableFunctionProvider.of( + new NoCompletionTimeoutAsyncTableFunction( + delayMillis, + returnsData, + lookupKeyIndexes, + physicalFieldCount)); + default: + throw new IllegalArgumentException("Unknown provider type: " + providerType); + } + } + + @Override + public DynamicTableSource copy() { + return new TimeoutAsyncLookupTableSource( + delayMillis, returnsData, providerType, physicalFieldCount); + } + + @Override + public String asSummaryString() { + return "TimeoutAsyncLookupTableSource(delayMillis=" + + delayMillis + + ", returnsData=" + + returnsData + + ", providerType=" + + providerType + + ")"; + } + } + + /** + * Custom exception raised by the non-standard {@code timeout(...)} hooks below. Tests assert + * this type to confirm that the user-defined timeout handler ran instead of falling back to + * {@link java.util.concurrent.TimeoutException}. + */ + public static class CustomLookupTimeoutException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public CustomLookupTimeoutException(String message) { + super(message); + } + } + + // --------------------------------------------------------------------------------------------- + // AsyncLookupFunction-based variant (new API) + // --------------------------------------------------------------------------------------------- + + /** An {@link AsyncLookupFunction} that sleeps before completing. */ + public static class TimeoutAsyncLookupFunction extends AsyncLookupFunction { + + private static final long serialVersionUID = 1L; + + private final long delayMillis; + private final boolean returnsData; + private final int[] lookupKeyIndexes; + private final int physicalFieldCount; + + private transient ExecutorService executor; + + public TimeoutAsyncLookupFunction( + long delayMillis, + boolean returnsData, + int[] lookupKeyIndexes, + int physicalFieldCount) { + this.delayMillis = delayMillis; + this.returnsData = returnsData; + this.lookupKeyIndexes = lookupKeyIndexes; + this.physicalFieldCount = physicalFieldCount; + } + + @Override + public void open(FunctionContext context) { + this.executor = Executors.newCachedThreadPool(); + } + + @Override + public CompletableFuture> asyncLookup(RowData keyRow) { + return CompletableFuture.supplyAsync( + () -> { + if (delayMillis > 0) { + try { + Thread.sleep(delayMillis); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + if (!returnsData) { + return Collections.emptyList(); + } + GenericRowData row = new GenericRowData(physicalFieldCount); + for (int i = 0; i < lookupKeyIndexes.length; i++) { + int outputIdx = lookupKeyIndexes[i]; + row.setField(outputIdx, ((GenericRowData) keyRow).getField(i)); + } + return Collections.singletonList(row); + }, + executor); + } + + public void timeout(CompletableFuture> future, Object... keys) { + future.completeExceptionally( + new CustomLookupTimeoutException( + "Custom timeout from TimeoutAsyncLookupFunction for keys " + + Arrays.toString(keys))); + } + + @Override + public void close() { + if (executor != null) { + executor.shutdownNow(); + } + } + } + + // --------------------------------------------------------------------------------------------- + // AsyncTableFunction-based variant (legacy API) + // --------------------------------------------------------------------------------------------- + + /** + * An {@link AsyncTableFunction} that mirrors {@link TimeoutAsyncLookupFunction} but uses the + * legacy reflection-based {@code eval(future, keys...)} signature. + */ + public static class TimeoutAsyncTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + private final long delayMillis; + private final boolean returnsData; + private final int[] lookupKeyIndexes; + private final int physicalFieldCount; + + private transient ExecutorService executor; + + public TimeoutAsyncTableFunction( + long delayMillis, + boolean returnsData, + int[] lookupKeyIndexes, + int physicalFieldCount) { + this.delayMillis = delayMillis; + this.returnsData = returnsData; + this.lookupKeyIndexes = lookupKeyIndexes; + this.physicalFieldCount = physicalFieldCount; + } + + @Override + public void open(FunctionContext context) { + this.executor = Executors.newCachedThreadPool(); + } + + public void eval(CompletableFuture> future, Object... keys) { + executor.submit( + () -> { + try { + if (delayMillis > 0) { + Thread.sleep(delayMillis); + } + if (!returnsData) { + future.complete(Collections.emptyList()); + return; + } + GenericRowData row = new GenericRowData(physicalFieldCount); + for (int i = 0; i < lookupKeyIndexes.length; i++) { + row.setField(lookupKeyIndexes[i], keys[i]); + } + future.complete(Collections.singletonList(row)); + } catch (Throwable t) { + future.completeExceptionally(t); + } + }); + } + + public void timeout(CompletableFuture> future, Object... keys) { + future.completeExceptionally( + new CustomLookupTimeoutException( + "Custom timeout from TimeoutAsyncTableFunction for keys " + + Arrays.toString(keys))); + } + + @Override + public void close() { + if (executor != null) { + executor.shutdownNow(); + } + } + } + + // --------------------------------------------------------------------------------------------- + // AsyncTableFunction-based variant WITHOUT a user-supplied timeout method + // --------------------------------------------------------------------------------------------- + + /** + * Same shape as {@link TimeoutAsyncTableFunction} but deliberately omits the {@code + * timeout(future, keys...)} method. Used to verify that the framework still works correctly + * when the user does not override the timeout — the operator should fall back to the default + * {@link java.util.concurrent.TimeoutException} instead of failing during codegen because the + * generated {@code timeout(...)} forwards to a non-existent method. + */ + public static class NoTimeoutAsyncTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + private final long delayMillis; + private final boolean returnsData; + private final int[] lookupKeyIndexes; + private final int physicalFieldCount; + + private transient ExecutorService executor; + + public NoTimeoutAsyncTableFunction( + long delayMillis, + boolean returnsData, + int[] lookupKeyIndexes, + int physicalFieldCount) { + this.delayMillis = delayMillis; + this.returnsData = returnsData; + this.lookupKeyIndexes = lookupKeyIndexes; + this.physicalFieldCount = physicalFieldCount; + } + + @Override + public void open(FunctionContext context) { + this.executor = Executors.newCachedThreadPool(); + } + + public void eval(CompletableFuture> future, Object... keys) { + executor.submit( + () -> { + try { + if (delayMillis > 0) { + Thread.sleep(delayMillis); + } + if (!returnsData) { + future.complete(Collections.emptyList()); + return; + } + GenericRowData row = new GenericRowData(physicalFieldCount); + for (int i = 0; i < lookupKeyIndexes.length; i++) { + row.setField(lookupKeyIndexes[i], keys[i]); + } + future.complete(Collections.singletonList(row)); + } catch (Throwable t) { + future.completeExceptionally(t); + } + }); + } + + // NOTE: Intentionally NO timeout(...) method declared here. + + @Override + public void close() { + if (executor != null) { + executor.shutdownNow(); + } + } + } + + // --------------------------------------------------------------------------------------------- + // AsyncTableFunction-based variant WITH an INCOMPATIBLE timeout(...) signature + // --------------------------------------------------------------------------------------------- + + /** + * Same shape as {@link TimeoutAsyncTableFunction} but the {@code timeout(...)} method's second + * parameter is {@link String} instead of the actual lookup key type ({@link Long} for a BIGINT + * key). The validation hook in {@link + * org.apache.flink.table.functions.UserDefinedFunctionHelper#validateAsyncTableFunctionTimeoutClass} + * must reject this signature with a {@link org.apache.flink.table.api.ValidationException + * ValidationException} during codegen. + */ + public static class IncompatibleTimeoutAsyncTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + private final long delayMillis; + private final boolean returnsData; + private final int[] lookupKeyIndexes; + private final int physicalFieldCount; + + private transient ExecutorService executor; + + public IncompatibleTimeoutAsyncTableFunction( + long delayMillis, + boolean returnsData, + int[] lookupKeyIndexes, + int physicalFieldCount) { + this.delayMillis = delayMillis; + this.returnsData = returnsData; + this.lookupKeyIndexes = lookupKeyIndexes; + this.physicalFieldCount = physicalFieldCount; + } + + @Override + public void open(FunctionContext context) { + this.executor = Executors.newCachedThreadPool(); + } + + public void eval(CompletableFuture> future, Object... keys) { + executor.submit( + () -> { + try { + if (delayMillis > 0) { + Thread.sleep(delayMillis); + } + if (!returnsData) { + future.complete(Collections.emptyList()); + return; + } + GenericRowData row = new GenericRowData(physicalFieldCount); + for (int i = 0; i < lookupKeyIndexes.length; i++) { + row.setField(lookupKeyIndexes[i], keys[i]); + } + future.complete(Collections.singletonList(row)); + } catch (Throwable t) { + future.completeExceptionally(t); + } + }); + } + + // Deliberately incompatible: lookup key is BIGINT (Long), but this expects String. + public void timeout(CompletableFuture> future, String wrongKey) { + future.completeExceptionally( + new CustomLookupTimeoutException( + "Should never be reached — codegen must reject this signature.")); + } + + @Override + public void close() { + if (executor != null) { + executor.shutdownNow(); + } + } + } + + // --------------------------------------------------------------------------------------------- + // AsyncTableFunction-based variant whose timeout(...) returns a DEFAULT row instead of failing + // --------------------------------------------------------------------------------------------- + + /** + * Same shape as {@link TimeoutAsyncTableFunction} but the {@code timeout(...)} method completes + * the result future with a single fallback row. The key columns echo the lookup key, and every + * non-key column is filled with the marker string {@code "DEFAULT"}. Used to verify that the + * runtime correctly forwards a {@code future.complete(...)} fallback emitted from the user's + * {@code timeout(...)} down the join pipeline, so the job finishes successfully and the + * timed-out records carry the default payload. + */ + public static class DefaultResultTimeoutAsyncTableFunction extends AsyncTableFunction { + + public static final String DEFAULT_VALUE_MARKER = "DEFAULT"; + + private static final long serialVersionUID = 1L; + + private final long delayMillis; + private final boolean returnsData; + private final int[] lookupKeyIndexes; + private final int physicalFieldCount; + + private transient ExecutorService executor; + + public DefaultResultTimeoutAsyncTableFunction( + long delayMillis, + boolean returnsData, + int[] lookupKeyIndexes, + int physicalFieldCount) { + this.delayMillis = delayMillis; + this.returnsData = returnsData; + this.lookupKeyIndexes = lookupKeyIndexes; + this.physicalFieldCount = physicalFieldCount; + } + + @Override + public void open(FunctionContext context) { + this.executor = Executors.newCachedThreadPool(); + } + + public void eval(CompletableFuture> future, Object... keys) { + executor.submit( + () -> { + try { + if (delayMillis > 0) { + Thread.sleep(delayMillis); + } + if (!returnsData) { + future.complete(Collections.emptyList()); + return; + } + GenericRowData row = new GenericRowData(physicalFieldCount); + for (int i = 0; i < lookupKeyIndexes.length; i++) { + row.setField(lookupKeyIndexes[i], keys[i]); + } + future.complete(Collections.singletonList(row)); + } catch (Throwable t) { + future.completeExceptionally(t); + } + }); + } + + /** + * Completes the future synchronously with a single default-valued row instead of an + * exception. The key columns echo the lookup keys; every other column receives the marker + * string so the test can distinguish a fallback row from a real lookup result. + */ + public void timeout(CompletableFuture> future, Object... keys) { + GenericRowData row = new GenericRowData(physicalFieldCount); + boolean[] isKeyColumn = new boolean[physicalFieldCount]; + for (int i = 0; i < lookupKeyIndexes.length; i++) { + row.setField(lookupKeyIndexes[i], keys[i]); + isKeyColumn[lookupKeyIndexes[i]] = true; + } + for (int i = 0; i < physicalFieldCount; i++) { + if (!isKeyColumn[i]) { + row.setField(i, StringData.fromString(DEFAULT_VALUE_MARKER)); + } + } + future.complete(Collections.singletonList(row)); + } + + @Override + public void close() { + if (executor != null) { + executor.shutdownNow(); + } + } + } + + // --------------------------------------------------------------------------------------------- + // AsyncTableFunction-based variant whose timeout(...) completes with an empty collection + // --------------------------------------------------------------------------------------------- + + /** + * Same shape as {@link TimeoutAsyncTableFunction} but the {@code timeout(...)} method completes + * the result future with an EMPTY collection. Used to verify that the LEFT OUTER lookup-join + * runtime preserves OUTER semantics under the user-defined timeout path — every timed-out left + * row must be padded with NULL on the right, not dropped, and the job must finish cleanly. + */ + public static class EmptyResultTimeoutAsyncTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + private final long delayMillis; + private final boolean returnsData; + private final int[] lookupKeyIndexes; + private final int physicalFieldCount; + + private transient ExecutorService executor; + + public EmptyResultTimeoutAsyncTableFunction( + long delayMillis, + boolean returnsData, + int[] lookupKeyIndexes, + int physicalFieldCount) { + this.delayMillis = delayMillis; + this.returnsData = returnsData; + this.lookupKeyIndexes = lookupKeyIndexes; + this.physicalFieldCount = physicalFieldCount; + } + + @Override + public void open(FunctionContext context) { + this.executor = Executors.newCachedThreadPool(); + } + + public void eval(CompletableFuture> future, Object... keys) { + executor.submit( + () -> { + try { + if (delayMillis > 0) { + Thread.sleep(delayMillis); + } + if (!returnsData) { + future.complete(Collections.emptyList()); + return; + } + GenericRowData row = new GenericRowData(physicalFieldCount); + for (int i = 0; i < lookupKeyIndexes.length; i++) { + row.setField(lookupKeyIndexes[i], keys[i]); + } + future.complete(Collections.singletonList(row)); + } catch (Throwable t) { + future.completeExceptionally(t); + } + }); + } + + public void timeout(CompletableFuture> future, Object... keys) { + future.complete(Collections.emptyList()); + } + + @Override + public void close() { + if (executor != null) { + executor.shutdownNow(); + } + } + } + + // --------------------------------------------------------------------------------------------- + // AsyncTableFunction-based variant with MULTIPLE timeout(...) overloads + // --------------------------------------------------------------------------------------------- + + /** + * Same shape as {@link TimeoutAsyncTableFunction} but declares TWO {@code timeout(...)} + * overloads — only the {@code (CompletableFuture, Object...)} one matches the lookup-key + * signature; the arity-3 overload exists purely to confirm that codegen resolves to the + * matching overload via Java compile-time overload selection on the generated {@code + * functionTerm.timeout(future, keys...)} call site (and that validation tolerates the extra + * non-matching method). The matching overload completes with a fallback row whose non-key + * columns carry the {@link #OVERLOAD_VALUE_MARKER} so the test can assert that it — and not the + * decoy overload — actually ran. + */ + public static class OverloadedTimeoutAsyncTableFunction extends AsyncTableFunction { + + public static final String OVERLOAD_VALUE_MARKER = "OVERLOAD"; + + private static final long serialVersionUID = 1L; + + private final long delayMillis; + private final boolean returnsData; + private final int[] lookupKeyIndexes; + private final int physicalFieldCount; + + private transient ExecutorService executor; + + public OverloadedTimeoutAsyncTableFunction( + long delayMillis, + boolean returnsData, + int[] lookupKeyIndexes, + int physicalFieldCount) { + this.delayMillis = delayMillis; + this.returnsData = returnsData; + this.lookupKeyIndexes = lookupKeyIndexes; + this.physicalFieldCount = physicalFieldCount; + } + + @Override + public void open(FunctionContext context) { + this.executor = Executors.newCachedThreadPool(); + } + + public void eval(CompletableFuture> future, Object... keys) { + executor.submit( + () -> { + try { + if (delayMillis > 0) { + Thread.sleep(delayMillis); + } + if (!returnsData) { + future.complete(Collections.emptyList()); + return; + } + GenericRowData row = new GenericRowData(physicalFieldCount); + for (int i = 0; i < lookupKeyIndexes.length; i++) { + row.setField(lookupKeyIndexes[i], keys[i]); + } + future.complete(Collections.singletonList(row)); + } catch (Throwable t) { + future.completeExceptionally(t); + } + }); + } + + /** Matching overload — must be picked by codegen's generated dispatch. */ + public void timeout(CompletableFuture> future, Object... keys) { + GenericRowData row = new GenericRowData(physicalFieldCount); + boolean[] isKeyColumn = new boolean[physicalFieldCount]; + for (int i = 0; i < lookupKeyIndexes.length; i++) { + row.setField(lookupKeyIndexes[i], keys[i]); + isKeyColumn[lookupKeyIndexes[i]] = true; + } + for (int i = 0; i < physicalFieldCount; i++) { + if (!isKeyColumn[i]) { + row.setField(i, StringData.fromString(OVERLOAD_VALUE_MARKER)); + } + } + future.complete(Collections.singletonList(row)); + } + + /** Decoy overload (wrong arity) — must NOT be invoked. */ + public void timeout( + CompletableFuture> future, Long key, Long extraArg) { + future.completeExceptionally( + new CustomLookupTimeoutException( + "Decoy (arity-3) overload must never run — picked: " + key)); + } + + @Override + public void close() { + if (executor != null) { + executor.shutdownNow(); + } + } + } + + // --------------------------------------------------------------------------------------------- + // AsyncTableFunction-based variant whose timeout(...) returns without completing the future + // --------------------------------------------------------------------------------------------- + + /** + * Same shape as {@link TimeoutAsyncTableFunction} but the {@code timeout(...)} method does + * NOTHING — it returns without completing the future synchronously. Used to verify that the + * codegen-emitted {@code isDone()} check on the lookup-join path fails fast with an {@link + * IllegalStateException} (symmetric to the correlate-path test for the same contract). + * + *

Constructor arguments are kept identical to the other {@code + * TimeoutAsyncLookupTableSource} variants so {@link TimeoutAsyncLookupTableSource} can + * instantiate it uniformly; only {@code delayMillis} is actually consulted (to keep the + * never-completing eval looping until the framework timer fires). + */ + public static class NoCompletionTimeoutAsyncTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + private final long delayMillis; + + private transient ExecutorService executor; + + public NoCompletionTimeoutAsyncTableFunction( + long delayMillis, + boolean returnsData, + int[] lookupKeyIndexes, + int physicalFieldCount) { + this.delayMillis = delayMillis; + } + + @Override + public void open(FunctionContext context) { + this.executor = Executors.newCachedThreadPool(); + } + + public void eval(CompletableFuture> future, Object... keys) { + // Schedule a sleep that outlives the configured timeout so timeout() is what runs. + executor.submit( + () -> { + try { + if (delayMillis > 0) { + Thread.sleep(delayMillis); + } + } catch (InterruptedException ignored) { + Thread.currentThread().interrupt(); + } + }); + } + + /** + * Intentionally returns without completing the future (no {@code complete}, no {@code + * completeExceptionally}, no throw). The codegen-emitted {@code isDone()} check is the only + * thing standing between this and a hung ResultHandler. + */ + public void timeout(CompletableFuture> future, Object... keys) { + // no-op + } + + @Override + public void close() { + if (executor != null) { + executor.shutdownNow(); + } + } + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncCorrelateTimeoutITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncCorrelateTimeoutITCase.java new file mode 100644 index 0000000000000..1638c10d7820e --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncCorrelateTimeoutITCase.java @@ -0,0 +1,359 @@ +/* + * 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.sql; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.ValidationException; +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.types.Row; +import org.apache.flink.util.CollectionUtil; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +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.TimeoutException; + +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * End-to-end coverage for the user-defined {@code timeout(...)} convention on {@link + * AsyncTableFunction} when invoked from a correlate (LATERAL TABLE) query, complementing the + * lookup-join coverage in {@code AsyncTableFunctionTimeoutITCase} / {@code + * AsyncLookupJoinTimeoutITCase}. Drives the correlate path through {@code + * AsyncCorrelateCodeGenerator} + {@code AsyncCorrelateRunner}, which previously discarded the + * generated timeout body. + * + *

Cases covered (mirroring the lookup-join test): + * + *

    + *
  • A valid {@code timeout(...)} that completes the future exceptionally surfaces the user's + * exception instead of the framework default {@link TimeoutException}. + *
  • A {@code timeout(...)} that completes with a default row routes that row downstream instead + * of failing the job. + *
  • An {@code AsyncTableFunction} without any {@code timeout(...)} method falls back to the + * framework default {@link TimeoutException} (codegen must not break). + *
  • An incompatible {@code timeout(...)} signature is rejected by codegen with a {@link + * ValidationException} before submission. + *
+ */ +class AsyncCorrelateTimeoutITCase { + + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @BeforeEach + void setup() { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + tEnv = + StreamTableEnvironment.create( + env, EnvironmentSettings.newInstance().inStreamingMode().build()); + tEnv.getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_MAX_CONCURRENT_OPERATIONS, 1); + tEnv.getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_ASYNC_TABLE_TIMEOUT, Duration.ofMillis(200)); + } + + @Test + void testCustomTimeoutFallbackForCorrelateQuery() { + tEnv.createTemporarySystemFunction("slow_func", new TimeoutThrowingTableFunction()); + createSourceAndSink(); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, t.s FROM source_t AS s,\n" + + " LATERAL TABLE(slow_func(s.id)) AS t(s)"); + + assertThatThrownBy(() -> result.await()) + .satisfies(anyCauseMatches(CustomCorrelateTimeoutException.class)); + } + + @Test + void testDefaultTimeoutWhenCorrelateUdfHasNoTimeout() { + tEnv.createTemporarySystemFunction("slow_func", new NoTimeoutTableFunction()); + createSourceAndSink(); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, t.s FROM source_t AS s,\n" + + " LATERAL TABLE(slow_func(s.id)) AS t(s)"); + + assertThatThrownBy(() -> result.await()).satisfies(anyCauseMatches(TimeoutException.class)); + } + + @Test + void testCustomTimeoutReturnsDefaultRowForCorrelateQuery() throws Exception { + tEnv.createTemporarySystemFunction("slow_func", new TimeoutDefaultRowTableFunction()); + createSourceOnly(); + + List rows = + CollectionUtil.iteratorToList( + tEnv.executeSql( + "SELECT s.id, t.s FROM source_t AS s,\n" + + " LATERAL TABLE(slow_func(s.id)) AS t(s)") + .collect()); + + // Source emits 1..3 — every row times out, so each yields exactly one DEFAULT row. + assertThat(rows).hasSize(3); + assertThat(rows) + .allSatisfy( + row -> + assertThat(row.getField(1)) + .isEqualTo( + TimeoutDefaultRowTableFunction + .DEFAULT_VALUE_MARKER)); + } + + @Test + void testCustomTimeoutReturnsEmptyCollectionForCorrelateQuery() throws Exception { + tEnv.createTemporarySystemFunction("slow_func", new TimeoutEmptyTableFunction()); + createSourceOnly(); + + // INNER (cross) correlate: timeout completing with an empty collection must produce + // zero output rows for each input — the left row is dropped, not padded. + List rows = + CollectionUtil.iteratorToList( + tEnv.executeSql( + "SELECT s.id, t.s FROM source_t AS s,\n" + + " LATERAL TABLE(slow_func(s.id)) AS t(s)") + .collect()); + + assertThat(rows).isEmpty(); + } + + @Test + void testCustomTimeoutReturnsMultipleRowsForCorrelateQuery() throws Exception { + tEnv.createTemporarySystemFunction("slow_func", new TimeoutMultiRowTableFunction()); + createSourceOnly(); + + // Source emits 1..3; each row times out and the user's timeout(...) emits two rows. + // Verifies that JoinedRowResultFuture correctly fans out the left row across the + // multi-row fallback (one-to-many correlate semantics under timeout). + List rows = + CollectionUtil.iteratorToList( + tEnv.executeSql( + "SELECT s.id, t.s FROM source_t AS s,\n" + + " LATERAL TABLE(slow_func(s.id)) AS t(s)") + .collect()); + + assertThat(rows).hasSize(6); + assertThat(rows) + .allSatisfy( + row -> + assertThat(row.getField(1)) + .isIn( + TimeoutMultiRowTableFunction.MARKER_A, + TimeoutMultiRowTableFunction.MARKER_B)); + } + + /** + * The user's {@code timeout(...)} must complete the future before it returns. Codegen + * enforces this by checking {@code delegates.getCompletableFuture().isDone()} after the call + * and forcing {@link IllegalStateException} if it's not — without that check, an + * incorrectly-written timeout (one that issues another async call and stores the future for a + * later callback, or simply forgets to complete it) would leave the operator's {@code + * ResultHandler} hanging until shutdown. This test drives that exact path: the UDF returns + * without touching the future, and the job must fail with the codegen-emitted {@code + * IllegalStateException}. + */ + @Test + void testTimeoutNotCompletingFutureFailsFast() { + tEnv.createTemporarySystemFunction("slow_func", new NoCompletionTimeoutTableFunction()); + createSourceAndSink(); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, t.s FROM source_t AS s,\n" + + " LATERAL TABLE(slow_func(s.id)) AS t(s)"); + + assertThatThrownBy(() -> result.await()) + .satisfies(anyCauseMatches(IllegalStateException.class, "synchronously")); + } + + @Test + void testIncompatibleTimeoutSignatureFailsFastForCorrelateQuery() { + tEnv.createTemporarySystemFunction("slow_func", new IncompatibleTimeoutTableFunction()); + createSourceAndSink(); + + // Codegen runs during executeSql planning, so ValidationException must surface here — + // before any job is submitted or task is initialised. + assertThatThrownBy( + () -> + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, t.s FROM source_t AS s,\n" + + " LATERAL TABLE(slow_func(s.id)) AS t(s)")) + .satisfies( + anyCauseMatches( + ValidationException.class, + IncompatibleTimeoutTableFunction.class.getName())) + .satisfies(anyCauseMatches(ValidationException.class, "CompletableFuture")) + .satisfies(anyCauseMatches(ValidationException.class, "Long")); + } + + private void createSourceAndSink() { + createSourceOnly(); + tEnv.executeSql( + "CREATE TABLE sink_t (id BIGINT, s STRING) WITH ('connector' = 'blackhole')"); + } + + private void createSourceOnly() { + tEnv.executeSql( + "CREATE TABLE source_t (id BIGINT) WITH (\n" + + " 'connector' = 'datagen',\n" + + " 'fields.id.kind' = 'sequence',\n" + + " 'fields.id.start' = '1',\n" + + " 'fields.id.end' = '3'\n" + + ")"); + } + + /** Custom exception raised by user-supplied {@code timeout(...)} hooks below. */ + public static class CustomCorrelateTimeoutException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public CustomCorrelateTimeoutException(String message) { + super(message); + } + } + + /** Never completes eval; timeout completes the future exceptionally. */ + public static class TimeoutThrowingTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + public void eval(CompletableFuture> future, Long key) { + // Never completes — only timeout() ever runs. + } + + public void timeout(CompletableFuture> future, Long key) { + future.completeExceptionally( + new CustomCorrelateTimeoutException( + "Custom timeout from TimeoutThrowingTableFunction for key " + key)); + } + } + + /** Never completes eval; timeout completes with a single DEFAULT row. */ + public static class TimeoutDefaultRowTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + public static final String DEFAULT_VALUE_MARKER = "DEFAULT"; + + public void eval(CompletableFuture> future, Long key) { + // Never completes — only timeout() ever runs. + } + + public void timeout(CompletableFuture> future, Long key) { + future.complete(Collections.singletonList(DEFAULT_VALUE_MARKER)); + } + } + + /** Never completes eval; timeout completes with an empty collection (zero rows). */ + public static class TimeoutEmptyTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + public void eval(CompletableFuture> future, Long key) { + // Never completes — only timeout() ever runs. + } + + public void timeout(CompletableFuture> future, Long key) { + future.complete(Collections.emptyList()); + } + } + + /** Never completes eval; timeout completes with two rows to exercise the fan-out path. */ + public static class TimeoutMultiRowTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + public static final String MARKER_A = "A"; + public static final String MARKER_B = "B"; + + public void eval(CompletableFuture> future, Long key) { + // Never completes — only timeout() ever runs. + } + + public void timeout(CompletableFuture> future, Long key) { + future.complete(Arrays.asList(MARKER_A, MARKER_B)); + } + } + + /** + * Never completes eval; timeout returns without completing (nor throwing). Simulates the + * canonical mis-use of the contract: user issues another async call and stores the future for a + * later callback. Codegen must detect the un-done future and fail-fast. + */ + public static class NoCompletionTimeoutTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + public void eval(CompletableFuture> future, Long key) { + // Never completes — only timeout() ever runs. + } + + public void timeout(CompletableFuture> future, Long key) { + // Intentionally do nothing — neither complete nor throw. The codegen-emitted + // isDone() check is the only thing standing between this and a hung ResultHandler. + } + } + + /** Never completes eval; declares NO timeout method (default fallback). */ + public static class NoTimeoutTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + public void eval(CompletableFuture> future, Long key) { + // Never completes — only the framework default timeout runs. + } + // NOTE: Intentionally NO timeout(...) method declared here. + } + + /** + * Declares an incompatible {@code timeout(...)} signature ({@code String} where the BIGINT key + * requires {@code Long}) — codegen must reject with a {@link ValidationException}. + */ + public static class IncompatibleTimeoutTableFunction extends AsyncTableFunction { + + private static final long serialVersionUID = 1L; + + public void eval(CompletableFuture> future, Long key) { + future.complete(Arrays.asList("blah", "foo")); + } + + public void timeout(CompletableFuture> future, String wrongKey) { + future.completeExceptionally( + new CustomCorrelateTimeoutException( + "Should never be reached — codegen must reject this signature.")); + } + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncLookupJoinTimeoutITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncLookupJoinTimeoutITCase.java new file mode 100644 index 0000000000000..f113b794cadd1 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncLookupJoinTimeoutITCase.java @@ -0,0 +1,302 @@ +/* + * 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.sql; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.planner.factories.TimeoutAsyncLookupTableFactory.CustomLookupTimeoutException; +import org.apache.flink.table.planner.factories.TimeoutAsyncLookupTableFactory.IncompatibleTimeoutAsyncTableFunction; +import org.apache.flink.table.planner.factories.TimeoutAsyncLookupTableFactory.ProviderType; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import java.time.Duration; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * End-to-end test that reproduces the timeout behaviour of {@code AsyncLookupJoinRunner} purely via + * Flink SQL. + * + *

Topology: + * + *

    + *
  • Source — {@code datagen} bounded sequence (records 1..N) + *
  • Dim — custom {@code timeout-async-lookup} connector with a configurable per-lookup sleep, + * exposing either an {@link org.apache.flink.table.functions.AsyncLookupFunction} or the + * legacy {@link org.apache.flink.table.functions.AsyncTableFunction} + *
  • Sink — {@code blackhole} + *
+ * + *

Both function variants expose a (non-standard) {@code timeout(...)} method that raises {@link + * CustomLookupTimeoutException}. The user-defined timeout handler runs in place of the framework + * default {@link java.util.concurrent.TimeoutException TimeoutException} regardless of whether the + * timeout is set via {@code table.exec.async-lookup.timeout} or a per-join {@code + * LOOKUP('timeout'=...)} hint. + * + *

How the user-side {@code timeout()} is reached: {@code FunctionCodeGenerator.scala} now + * appends a {@code timeout(input, ResultFuture)} override to the generated {@code AsyncFunction} + * subclass, reusing the same body as {@code asyncInvoke} with {@code eval} rewritten to {@code + * timeout}. So when the {@code AsyncWaitOperator} fires the timeout timer, it calls into the + * generated function's {@code timeout(...)}, which delegates to the wrapped UDF's {@code + * timeout(future, keys...)}. The legacy {@link org.apache.flink.table.functions.AsyncTableFunction + * AsyncTableFunction} variant matches that signature directly; the new {@link + * org.apache.flink.table.functions.AsyncLookupFunction AsyncLookupFunction} variant exposes the + * same shape via the inherited reflective {@code eval(future, keys...)} machinery (see the + * non-standard {@code timeout(future, keys...)} method declared on the test UDF for symmetry). + * + *

Each parameterized timeout test runs once per {@link ProviderType}. {@link + * #testNoTimeoutCompletesSuccessfully()} stays as a single sanity case ensuring the scaffolding + * does not accidentally raise the custom exception when no timeout fires. + */ +class AsyncLookupJoinTimeoutITCase { + + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @BeforeEach + void setup() { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + tEnv = + StreamTableEnvironment.create( + env, EnvironmentSettings.newInstance().inStreamingMode().build()); + } + + /** + * Configures the async-lookup timeout via {@code table.exec.async-lookup.timeout} (200 ms) and + * forces every record to time out by setting the dim's per-lookup delay to 3 s. Asserts that + * the user-defined {@code timeout(...)} on the UDF runs and surfaces {@link + * CustomLookupTimeoutException} instead of the framework default {@code TimeoutException}. + * + *

Runs once per {@link ProviderType} — new {@code AsyncLookupFunction} and legacy {@code + * AsyncTableFunction}. + */ + @ParameterizedTest(name = "providerType={0}") + @EnumSource( + value = ProviderType.class, + names = {"ASYNC_LOOKUP_FUNCTION", "ASYNC_TABLE_FUNCTION"}) + void testCustomTimeoutViaConfig(ProviderType providerType) { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables(/* lookupDelay */ "3s", /* returnsData */ false, providerType); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id"); + + assertThatThrownBy(() -> result.await()) + .satisfies(anyCauseMatches(CustomLookupTimeoutException.class)); + } + + /** + * Same assertion as {@link #testCustomTimeoutViaConfig(ProviderType)}, but the per-job global + * timeout is permissive (30 s) and the 200 ms timeout is pinned per-join via the {@code + * LOOKUP('timeout'=...)} hint. Verifies that the hint path also routes through the user's + * custom {@code timeout(...)}. + * + *

Runs once per {@link ProviderType}. + */ + @ParameterizedTest(name = "providerType={0}") + @EnumSource( + value = ProviderType.class, + names = {"ASYNC_LOOKUP_FUNCTION", "ASYNC_TABLE_FUNCTION"}) + void testCustomTimeoutViaLookupHint(ProviderType providerType) { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofSeconds(30)); + + createTables(/* lookupDelay */ "3s", /* returnsData */ false, providerType); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT /*+ LOOKUP('table'='d', 'async'='true', 'timeout'='200 ms') */\n" + + " s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id"); + + assertThatThrownBy(() -> result.await()) + .satisfies(anyCauseMatches(CustomLookupTimeoutException.class)); + } + + /** + * Verifies that an {@link org.apache.flink.table.functions.AsyncTableFunction} works correctly + * even when the user does not declare a {@code timeout(...)} method. Expected behaviour: the + * operator falls back to the framework default {@link java.util.concurrent.TimeoutException + * TimeoutException} just like before the custom-timeout feature was added — i.e. the absence of + * a user-supplied timeout handler must not crash codegen. + * + *

Currently FAILS — this is expected. {@code FunctionCodeGenerator.scala} emits a + * {@code timeout(input, ResultFuture)} override unconditionally and forwards to {@code + * udf.timeout(future, keys...)}, so when the UDF does not provide one the generated class fails + * to compile (or fails at first timeout dispatch). The fix is in progress: the codegen needs to + * detect whether the UDF declares a {@code timeout(...)} and only emit the override when it + * does (or wrap the call so the missing method falls back to the default behaviour). Once that + * lands, this test should turn green without any change. + */ + @Test + void testFallbackToDefaultTimeoutWhenUserTimeoutAbsent() { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables( + /* lookupDelay */ "3s", + /* returnsData */ false, + ProviderType.ASYNC_TABLE_FUNCTION_WITHOUT_TIMEOUT); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id"); + + // Without a user-defined timeout method, the framework default must fire — surfacing + // java.util.concurrent.TimeoutException, NOT CustomLookupTimeoutException and NOT a + // codegen-compilation error. + assertThatThrownBy(() -> result.await()).satisfies(anyCauseMatches(TimeoutException.class)); + } + + /** + * Verifies that a UDF declaring a {@code timeout(...)} method whose parameter list is + * incompatible with the lookup keys is rejected by codegen with a {@link ValidationException} + * at SQL compilation time (i.e. before the job is ever submitted). The error message must + * mention the UDF's fully-qualified class name plus the expected and actual signatures so a + * user can locate the bad method quickly. + */ + @Test + void testIncompatibleTimeoutSignatureFailsFast() { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables( + /* lookupDelay */ "3s", + /* returnsData */ false, + ProviderType.ASYNC_TABLE_FUNCTION_WITH_INCOMPATIBLE_TIMEOUT); + + // Codegen runs during executeSql planning, so ValidationException must surface here — + // before any job is submitted or task is initialised. + assertThatThrownBy( + () -> + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id")) + .satisfies( + anyCauseMatches( + ValidationException.class, + IncompatibleTimeoutAsyncTableFunction.class.getName())) + .satisfies(anyCauseMatches(ValidationException.class, "CompletableFuture")) + .satisfies(anyCauseMatches(ValidationException.class, "Long")); + } + + /** + * Sanity check (non-parameterized): when the lookup delay (50 ms) is well below the configured + * timeout (10 s), no timeout fires and the job completes cleanly. Used to confirm that the test + * scaffolding itself doesn't accidentally raise {@link CustomLookupTimeoutException}. + */ + @Test + void testNoTimeoutCompletesSuccessfully() { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofSeconds(10)); + + createTables( + /* lookupDelay */ "50ms", + /* returnsData */ true, + ProviderType.ASYNC_LOOKUP_FUNCTION); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id"); + + assertThat(result).isNotNull(); + assertThatCode(() -> result.await()).doesNotThrowAnyException(); + } + + private void createTables(String lookupDelay, boolean returnsData, ProviderType providerType) { + // Bounded datagen source — small sequence so the test finishes quickly. + tEnv.executeSql( + "CREATE TABLE source_t (\n" + + " id BIGINT,\n" + + " proc AS PROCTIME()\n" + + ") WITH (\n" + + " 'connector' = 'datagen',\n" + + " 'fields.id.kind' = 'sequence',\n" + + " 'fields.id.start' = '1',\n" + + " 'fields.id.end' = '5'\n" + + ")"); + + // Custom dim connector exposing async lookup with a configurable delay and provider type. + tEnv.executeSql( + "CREATE TABLE dim_t (\n" + + " id BIGINT,\n" + + " name STRING,\n" + + " PRIMARY KEY (id) NOT ENFORCED\n" + + ") WITH (\n" + + " 'connector' = 'timeout-async-lookup',\n" + + " 'lookup-delay' = '" + + lookupDelay + + "',\n" + + " 'returns-data' = '" + + returnsData + + "',\n" + + " 'provider-type' = '" + + providerType.name() + + "'\n" + + ")"); + + // Blackhole sink — discards everything, satisfies INSERT INTO without external state. + tEnv.executeSql( + "CREATE TABLE sink_t (\n" + + " id BIGINT,\n" + + " name STRING\n" + + ") WITH (\n" + + " 'connector' = 'blackhole'\n" + + ")"); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncTableFunctionTimeoutITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncTableFunctionTimeoutITCase.java new file mode 100644 index 0000000000000..94a41ba658efc --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/AsyncTableFunctionTimeoutITCase.java @@ -0,0 +1,349 @@ +/* + * 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.sql; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.ValidationException; +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.factories.TimeoutAsyncLookupTableFactory.CustomLookupTimeoutException; +import org.apache.flink.table.planner.factories.TimeoutAsyncLookupTableFactory.DefaultResultTimeoutAsyncTableFunction; +import org.apache.flink.table.planner.factories.TimeoutAsyncLookupTableFactory.IncompatibleTimeoutAsyncTableFunction; +import org.apache.flink.table.planner.factories.TimeoutAsyncLookupTableFactory.OverloadedTimeoutAsyncTableFunction; +import org.apache.flink.table.planner.factories.TimeoutAsyncLookupTableFactory.ProviderType; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.List; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * End-to-end coverage for the user-defined {@code timeout(...)} convention declared on {@link + * AsyncTableFunction}. The custom-timeout contract lives on {@code AsyncTableFunction} itself + * (signature lookup, fail-fast validation, default fallback), so these cases exercise the three + * contract branches with UDFs that extend {@code AsyncTableFunction} directly — without relying on + * the convenience subclass {@code AsyncLookupFunction}. + * + *

Cases covered: + * + *

    + *
  • A valid {@code timeout(...)} on {@code AsyncTableFunction} runs and replaces the framework + * default {@link TimeoutException} with the user's exception. + *
  • A {@code timeout(...)} that completes the future with a default row routes that row + * downstream instead of failing the job. + *
  • An {@code AsyncTableFunction} without any {@code timeout(...)} method falls back to the + * framework default {@link TimeoutException} (no codegen failure). + *
  • An {@code AsyncTableFunction} declaring an incompatible {@code timeout(...)} signature is + * rejected by codegen with {@link ValidationException} before submission. + *
+ * + *

The shared {@code AsyncLookupFunction} coverage (parameterized over both providers) lives in + * {@code AsyncLookupJoinTimeoutITCase}; this class intentionally narrows the surface to the {@code + * AsyncTableFunction} contract owner. + */ +class AsyncTableFunctionTimeoutITCase { + + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @BeforeEach + void setup() { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + tEnv = + StreamTableEnvironment.create( + env, EnvironmentSettings.newInstance().inStreamingMode().build()); + } + + /** + * An {@link AsyncTableFunction} subclass that declares a valid {@code + * timeout(CompletableFuture, Object...)} must have its handler invoked on timeout, surfacing + * the user-defined {@link CustomLookupTimeoutException} instead of the framework default {@link + * TimeoutException}. + */ + @Test + void testCustomTimeoutFallbackForAsyncTableFunction() { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables(/* lookupDelay */ "3s", ProviderType.ASYNC_TABLE_FUNCTION); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id"); + + assertThatThrownBy(() -> result.await()) + .satisfies(anyCauseMatches(CustomLookupTimeoutException.class)); + } + + /** + * An {@link AsyncTableFunction} subclass that declares no {@code timeout(...)} method must keep + * the legacy behaviour: the framework default {@link TimeoutException} fires. Codegen must not + * fail just because the user omitted the handler. + */ + @Test + void testDefaultTimeoutWhenAsyncTableFunctionTimeoutAbsent() { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables(/* lookupDelay */ "3s", ProviderType.ASYNC_TABLE_FUNCTION_WITHOUT_TIMEOUT); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id"); + + assertThatThrownBy(() -> result.await()).satisfies(anyCauseMatches(TimeoutException.class)); + } + + /** + * When the user's {@code timeout(...)} method calls {@code future.complete()} + * instead of {@code completeExceptionally(...)}, the job must finish successfully and every + * timed-out lookup must surface the default payload downstream. Drives the source 1..5 through + * a forced 3-second lookup delay against a 200 ms timeout, then collects the join result and + * asserts that all five rows carry the {@code "DEFAULT"} marker supplied by the {@code + * timeout(...)} fallback. + */ + @Test + void testCustomTimeoutReturnsDefaultRow() throws Exception { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables( + /* lookupDelay */ "3s", + ProviderType.ASYNC_TABLE_FUNCTION_WITH_DEFAULT_RESULT_TIMEOUT); + + List rows = + CollectionUtil.iteratorToList( + tEnv.executeSql( + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc" + + " AS d\n" + + "ON s.id = d.id") + .collect()); + + // source_t is a bounded datagen sequence 1..5, so we expect five rows; every one of them + // must have been routed through the user's timeout(...) and carry the DEFAULT marker. + assertThat(rows).hasSize(5); + assertThat(rows) + .allSatisfy( + row -> + assertThat(row.getField(1)) + .isEqualTo( + DefaultResultTimeoutAsyncTableFunction + .DEFAULT_VALUE_MARKER)); + } + + /** + * Verifies LEFT OUTER lookup-join semantics under the user-defined timeout path: when the + * user's {@code timeout(...)} completes with an EMPTY collection, every timed-out left row must + * still flow downstream with the right side padded with NULL — not dropped. Drives the source + * 1..5 through a 3s lookup delay against a 200 ms timeout and asserts that all five rows arrive + * with {@code d.name == null}. + */ + @Test + void testCustomTimeoutEmptyCollectionPadsNullsForLeftOuterJoin() throws Exception { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables( + /* lookupDelay */ "3s", + ProviderType.ASYNC_TABLE_FUNCTION_WITH_EMPTY_RESULT_TIMEOUT); + + List rows = + CollectionUtil.iteratorToList( + tEnv.executeSql( + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc" + + " AS d\n" + + "ON s.id = d.id") + .collect()); + + assertThat(rows).hasSize(5); + assertThat(rows).allSatisfy(row -> assertThat(row.getField(1)).isNull()); + } + + /** + * Verifies that a UDF declaring multiple {@code timeout(...)} overloads compiles and the + * matching overload (mirroring the {@code eval(future, keys...)} signature) is the one that + * actually runs at timeout. Decoy overloads — wrong parameter type and wrong arity — must not + * be invoked; if either were picked the future would complete exceptionally with {@link + * CustomLookupTimeoutException} and the assertion would fail. The matching overload completes + * with a fallback row carrying {@link + * OverloadedTimeoutAsyncTableFunction#OVERLOAD_VALUE_MARKER}, so finishing cleanly with that + * marker proves the right method was dispatched. + */ + @Test + void testTimeoutMethodOverloadResolvesToMatchingSignature() throws Exception { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables( + /* lookupDelay */ "3s", ProviderType.ASYNC_TABLE_FUNCTION_WITH_OVERLOADED_TIMEOUT); + + List rows = + CollectionUtil.iteratorToList( + tEnv.executeSql( + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc" + + " AS d\n" + + "ON s.id = d.id") + .collect()); + + assertThat(rows).hasSize(5); + assertThat(rows) + .allSatisfy( + row -> + assertThat(row.getField(1)) + .isEqualTo( + OverloadedTimeoutAsyncTableFunction + .OVERLOAD_VALUE_MARKER)); + } + + /** + * The user's {@code timeout(...)} must complete the future before it returns. Codegen + * enforces this on the lookup-join path the same way as on the correlate path by checking + * {@code delegates.getCompletableFuture().isDone()} after the call and forcing {@link + * IllegalStateException} if it's not — symmetric coverage to {@code + * AsyncCorrelateTimeoutITCase#testTimeoutNotCompletingFutureFailsFast}. The UDF here returns + * without touching the future, and the job must fail with that codegen-emitted exception. + */ + @Test + void testLookupTimeoutNotCompletingFutureFailsFast() { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables( + /* lookupDelay */ "3s", + ProviderType.ASYNC_TABLE_FUNCTION_WITH_NO_COMPLETION_TIMEOUT); + + TableResult result = + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id"); + + assertThatThrownBy(() -> result.await()) + .satisfies(anyCauseMatches(IllegalStateException.class, "synchronously")); + } + + /** + * An {@link AsyncTableFunction} subclass declaring a {@code timeout(...)} whose parameter list + * is incompatible with the lookup keys (here: {@code String} where the BIGINT key requires + * {@code Long}) must be rejected by codegen with a {@link ValidationException} carrying the + * UDF's fully-qualified class name plus the expected and actual signatures. + */ + @Test + void testIncompatibleTimeoutSignatureOnAsyncTableFunctionFailsFast() { + tEnv.getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT, + Duration.ofMillis(200)); + + createTables( + /* lookupDelay */ "3s", + ProviderType.ASYNC_TABLE_FUNCTION_WITH_INCOMPATIBLE_TIMEOUT); + + // Codegen runs during executeSql planning, so ValidationException must surface here — + // before any job is submitted or task is initialised. + assertThatThrownBy( + () -> + tEnv.executeSql( + "INSERT INTO sink_t\n" + + "SELECT s.id, d.name FROM source_t AS s\n" + + "LEFT JOIN dim_t FOR SYSTEM_TIME AS OF s.proc AS d\n" + + "ON s.id = d.id")) + .satisfies( + anyCauseMatches( + ValidationException.class, + IncompatibleTimeoutAsyncTableFunction.class.getName())) + .satisfies(anyCauseMatches(ValidationException.class, "CompletableFuture")) + .satisfies(anyCauseMatches(ValidationException.class, "Long")); + } + + private void createTables(String lookupDelay, ProviderType providerType) { + // Bounded datagen source — small sequence so the test finishes quickly. + tEnv.executeSql( + "CREATE TABLE source_t (\n" + + " id BIGINT,\n" + + " proc AS PROCTIME()\n" + + ") WITH (\n" + + " 'connector' = 'datagen',\n" + + " 'fields.id.kind' = 'sequence',\n" + + " 'fields.id.start' = '1',\n" + + " 'fields.id.end' = '5'\n" + + ")"); + + // Dim connector backed by an AsyncTableFunction variant selected via provider-type. + // returns-data=false keeps the test focused on timeout dispatch. + tEnv.executeSql( + "CREATE TABLE dim_t (\n" + + " id BIGINT,\n" + + " name STRING,\n" + + " PRIMARY KEY (id) NOT ENFORCED\n" + + ") WITH (\n" + + " 'connector' = 'timeout-async-lookup',\n" + + " 'lookup-delay' = '" + + lookupDelay + + "',\n" + + " 'returns-data' = 'false',\n" + + " 'provider-type' = '" + + providerType.name() + + "'\n" + + ")"); + + // Blackhole sink — discards everything, satisfies INSERT INTO without external state. + tEnv.executeSql( + "CREATE TABLE sink_t (\n" + + " id BIGINT,\n" + + " name STRING\n" + + ") WITH (\n" + + " 'connector' = 'blackhole'\n" + + ")"); + } +} diff --git a/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory index 30b07f2bd1e97..5f055635a8ae7 100644 --- a/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-table/flink-table-planner/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -23,4 +23,5 @@ org.apache.flink.table.planner.factories.TestSupportsStagingTableFactory org.apache.flink.table.planner.factories.TestProcedureCatalogFactory org.apache.flink.table.planner.utils.TestSimpleDynamicTableSourceFactory org.apache.flink.table.planner.factories.TestValuesModelFactory -org.apache.flink.table.planner.factories.PlainTestCatalogFactory \ No newline at end of file +org.apache.flink.table.planner.factories.PlainTestCatalogFactory +org.apache.flink.table.planner.factories.TimeoutAsyncLookupTableFactory diff --git a/flink-table/flink-table-planner/src/test/resources/log4j2-test.properties b/flink-table/flink-table-planner/src/test/resources/log4j2-test.properties index 148ab56eea455..4255d7815a4ee 100644 --- a/flink-table/flink-table-planner/src/test/resources/log4j2-test.properties +++ b/flink-table/flink-table-planner/src/test/resources/log4j2-test.properties @@ -18,7 +18,7 @@ # Set root logger level to OFF to not flood build logs # set manually to INFO for debugging purposes -rootLogger.level = OFF +rootLogger.level = INFO rootLogger.appenderRef.test.ref = TestLogger appender.testlogger.name = TestLogger @@ -32,3 +32,6 @@ appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n #logger.testlogger.name =org.apache.flink.table.planner.codegen #logger.testlogger.level = TRACE #logger.testlogger.appenderRefs = TestLogger +loggers = compileutils +logger.compileutils.name = org.apache.flink.table.runtime.generated.CompileUtils +logger.compileutils.level = DEBUG 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 index bbc4e2c1929b9..2ac66a5539892 100644 --- 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 @@ -78,6 +78,20 @@ public void asyncInvoke(RowData input, ResultFuture resultFuture) throw } } + @Override + public void timeout(RowData input, ResultFuture resultFuture) throws Exception { + // Forward to the user UDF's timeout(...) so the FLIP-498 custom-timeout contract reaches + // correlate queries the same way it reaches lookup join. + try { + JoinedRowResultFuture outResultFuture = + new JoinedRowResultFuture(input, resultFuture, fetcherConverter); + + fetcher.timeout(input, outResultFuture); + } catch (Throwable t) { + resultFuture.completeExceptionally(t); + } + } + @Override public void close() throws Exception { super.close(); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFuture.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFuture.java index 9d2885ba82dc9..e745eca01a80c 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFuture.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFuture.java @@ -58,9 +58,22 @@ public DelegatingAsyncTableResultFuture( public void accept(Collection outs, Throwable throwable) { if (throwable != null) { delegatedResultFuture.completeExceptionally(throwable); - } else { - delegatedResultFuture.complete(wrapFunction.apply(outs)); + return; } + // wrapFunction may throw (e.g. NPE on a null element, ClassCastException on a wrong + // payload type). Users typically complete the CompletableFuture from a callback running + // on their own async client's thread (a Netty / HTTP / RPC worker), so without this catch + // the exception would propagate on that user thread and the delegated ResultFuture would + // never be completed, leaving the AsyncWaitOperator's ResultHandler hanging forever. + // Forward the failure so the operator can surface it through the normal error path. + final Collection wrapped; + try { + wrapped = wrapFunction.apply(outs); + } catch (Throwable t) { + delegatedResultFuture.completeExceptionally(t); + return; + } + delegatedResultFuture.complete(wrapped); } private Collection wrapInternal(Collection outs) { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinRunner.java index 6b34c99f4a766..89db322509a22 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinRunner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinRunner.java @@ -43,6 +43,7 @@ import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; /** The async join runner to lookup the dimension table. */ public class AsyncLookupJoinRunner extends AbstractAsyncFunctionRunner { @@ -124,6 +125,9 @@ public void open(OpenContext openContext) throws Exception { @Override public void asyncInvoke(RowData input, ResultFuture resultFuture) throws Exception { JoinedRowResultFuture outResultFuture = resultFutureBuffer.take(); + if (!outResultFuture.inuse.compareAndSet(false, true)) { + throw new IllegalStateException(); + } // the input row is copied when object reuse in AsyncWaitOperator outResultFuture.reset(input, resultFuture); @@ -144,6 +148,50 @@ public TableFunctionResultFuture createFetcherResultFuture(Configuratio return resultFuture; } + @Override + public void timeout(RowData input, ResultFuture resultFuture) throws Exception { + // Find and discard the in-flight future bound to this input row so that any late + // completion from the underlying fetcher is ignored. The generated fetcher's own + // timeout method (rendered when the user UDF provides one) decides how to complete + // the new outResultFuture below; if no user timeout method is present, the default + // AsyncFunction.timeout raises a TimeoutException as before. + // + // Reference equality on leftRow is intentional: AsyncWaitOperator passes the same + // RowData instance to both asyncInvoke and timeout for a given record (the operator + // already deep-copies under object reuse). + JoinedRowResultFuture currentFuture = null; + for (JoinedRowResultFuture f : allResultFutures) { + if (f.leftRow == input) { + currentFuture = f; + break; + } + } + if (currentFuture == null || !currentFuture.inuse.compareAndSet(true, false)) { + // current future is already completed and reused + return; + } + allResultFutures.remove(currentFuture); + currentFuture.close(); + + // Route through join pipeline via new JoinedRowResultFuture + JoinedRowResultFuture outResultFuture = + new JoinedRowResultFuture( + resultFutureBuffer, + createFetcherResultFuture(new Configuration()), + fetcherConverter, + isLeftOuterJoin, + rightRowSerializer.getArity()); + outResultFuture.inuse.set(true); + outResultFuture.reset(input, resultFuture); + allResultFutures.add(outResultFuture); + + try { + fetcher.timeout(input, outResultFuture); + } catch (Throwable t) { + outResultFuture.completeExceptionally(t); + } + } + @Override public void close() throws Exception { super.close(); @@ -181,12 +229,14 @@ public List getAllResultFutures() { */ private static final class JoinedRowResultFuture implements ResultFuture { + private final AtomicBoolean inuse = new AtomicBoolean(false); + private final BlockingQueue resultFutureBuffer; private final TableFunctionResultFuture joinConditionResultFuture; private final DataStructureConverter resultConverter; private final boolean isLeftOuterJoin; - private final DelegateResultFuture delegate; + private final DelegateResultFuture delegate; private final GenericRowData nullRow; private RowData leftRow; @@ -202,7 +252,7 @@ private JoinedRowResultFuture( this.joinConditionResultFuture = joinConditionResultFuture; this.resultConverter = resultConverter; this.isLeftOuterJoin = isLeftOuterJoin; - this.delegate = new DelegateResultFuture(); + this.delegate = new DelegateResultFuture<>(); this.nullRow = new GenericRowData(rightArity); } @@ -217,6 +267,10 @@ public void reset(RowData row, ResultFuture realOutput) { @Override @SuppressWarnings({"unchecked", "rawtypes"}) public void complete(Collection result) { + if (!inuse.compareAndSet(true, false)) { + return; + } + Collection rowDataCollection; if (resultConverter.isIdentityConversion()) { rowDataCollection = (Collection) result; @@ -231,9 +285,14 @@ public void complete(Collection result) { // the filtered result will be routed to the delegateCollector try { joinConditionResultFuture.complete(rowDataCollection); + if (delegate.isCompletedExceptionally) { + // inuse is changed, this.completeExceptionally should not be called by delegate + realOutput.completeExceptionally(delegate.error); + return; + } } catch (Throwable t) { // we should catch the exception here to let the framework know - completeExceptionally(t); + realOutput.completeExceptionally(t); return; } @@ -258,12 +317,15 @@ public void complete(Collection result) { // again before outRows in the collector is not consumed. resultFutureBuffer.put(this); } catch (InterruptedException e) { - completeExceptionally(e); + realOutput.completeExceptionally(e); } } @Override public void completeExceptionally(Throwable error) { + if (!inuse.compareAndSet(true, false)) { + return; + } realOutput.completeExceptionally(error); } @@ -279,33 +341,40 @@ public void complete(CollectionSupplier supplier) { public void close() throws Exception { joinConditionResultFuture.close(); } + } - private final class DelegateResultFuture implements ResultFuture { + private static final class DelegateResultFuture implements ResultFuture { - private Collection collection; + private Collection collection; + private boolean isCompletedExceptionally = false; + private Throwable error; - public void reset() { - this.collection = null; - } + private DelegateResultFuture() {} - @Override - public void complete(Collection result) { - this.collection = result; - } + public void reset() { + this.collection = null; + this.error = null; + this.isCompletedExceptionally = false; + } - @Override - public void completeExceptionally(Throwable error) { - JoinedRowResultFuture.this.completeExceptionally(error); - } + @Override + public void complete(Collection result) { + this.collection = result; + } - /** - * Unsupported, because the containing classes are AsyncFunctions which don't have - * access to the mailbox to invoke from the caller thread. - */ - @Override - public void complete(CollectionSupplier supplier) { - throw new UnsupportedOperationException(); - } + @Override + public void completeExceptionally(Throwable error) { + this.isCompletedExceptionally = true; + this.error = error; + } + + /** + * Unsupported, because the containing classes are AsyncFunctions which don't have access to + * the mailbox to invoke from the caller thread. + */ + @Override + public void complete(CollectionSupplier supplier) { + throw new UnsupportedOperationException(); } } } 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 index 1377aab907830..d03ee7854b2ed 100644 --- 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 @@ -44,10 +44,12 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -109,6 +111,62 @@ void testException() throws Exception { .hasMessageContaining("Other Error!"); } + @Test + void testTimeoutForwardsToFetcher() throws Exception { + TestResultFuture resultFuture = new TestResultFuture(); + AsyncCorrelateRunner runner = + new AsyncCorrelateRunner( + new GeneratedFunctionWrapper<>(new TimeoutFallbackFunction()), + createConverter(RowType.of(new IntType()))); + runner.setRuntimeContext(createRuntimeContext()); + runner.open((OpenContext) null); + + runner.timeout(GenericRowData.of(7), resultFuture); + + Collection rows = resultFuture.getResult().get(); + assertThat(rows) + .containsExactly(new JoinedRowData(GenericRowData.of(7), GenericRowData.of(99))); + } + + @Test + void testTimeoutWithoutOverrideUsesDefaultTimeoutException() 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); + + // ImmediateCallbackFunction does not override timeout(), so the default AsyncFunction + // implementation propagates TimeoutException to the result future. + runner.timeout(GenericRowData.of(1), resultFuture); + + assertThatThrownBy(() -> resultFuture.getResult().get()) + .isInstanceOf(ExecutionException.class) + .cause() + .isInstanceOf(TimeoutException.class); + } + + @Test + void testTimeoutFetcherException() throws Exception { + TestResultFuture resultFuture = new TestResultFuture(); + AsyncCorrelateRunner runner = + new AsyncCorrelateRunner( + new GeneratedFunctionWrapper<>(new TimeoutThrowingFunction()), + createConverter(RowType.of(new IntType()))); + runner.setRuntimeContext(createRuntimeContext()); + runner.open((OpenContext) null); + + runner.timeout(GenericRowData.of(1), resultFuture); + + assertThatThrownBy(() -> resultFuture.getResult().get()) + .isInstanceOf(ExecutionException.class) + .cause() + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("timeout boom"); + } + private RuntimeContext createRuntimeContext() { return new RuntimeUDFContext( new TaskInfoImpl("", 1, 0, 1, 0), @@ -164,6 +222,32 @@ public void asyncInvoke(RowData input, ResultFuture resultFuture) throws } } + /** Function whose timeout(...) override completes the future with a fallback row. */ + public static class TimeoutFallbackFunction implements AsyncFunction { + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) { + // Never completes — the test only drives timeout() so this branch is unused. + } + + @Override + public void timeout(RowData input, ResultFuture resultFuture) { + resultFuture.complete(Collections.singletonList(Row.of(99))); + } + } + + /** Function whose timeout(...) override throws — verifies the catch in timeout(). */ + public static class TimeoutThrowingFunction implements AsyncFunction { + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) {} + + @Override + public void timeout(RowData input, ResultFuture resultFuture) { + throw new RuntimeException("timeout boom"); + } + } + /** Test result future. */ public static final class TestResultFuture implements ResultFuture { diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFutureTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFutureTest.java new file mode 100644 index 0000000000000..6b1d92d3c0b55 --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/correlate/async/DelegatingAsyncTableResultFutureTest.java @@ -0,0 +1,130 @@ +/* + * 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.streaming.api.functions.async.CollectionSupplier; +import org.apache.flink.streaming.api.functions.async.ResultFuture; + +import org.junit.jupiter.api.Test; + +import java.util.AbstractCollection; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicReference; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests {@link DelegatingAsyncTableResultFuture}. */ +class DelegatingAsyncTableResultFutureTest { + + @Test + void wrapExceptionIsForwardedToDelegatedResultFuture() { + // needsWrapping=true + isInternalResultType=false → wrapExternal, which calls Row.of on + // every element. The non-public, non-static `wrapExternal` does NOT itself throw on + // null elements (Row.of allows them), so we drive the wrap failure by completing the + // future with a Collection whose iteration throws. This proves the new try/catch in + // accept() forwards the failure as completeExceptionally rather than letting it + // propagate on the completing thread and stranding the ResultFuture. + CapturingResultFuture downstream = new CapturingResultFuture(); + DelegatingAsyncTableResultFuture bridge = + new DelegatingAsyncTableResultFuture( + downstream, /* needsWrapping */ true, /* isInternalResultType */ false); + + RuntimeException boom = new RuntimeException("wrap boom"); + bridge.getCompletableFuture().complete(new ThrowingOnIterateCollection(boom)); + + assertThat(downstream.error.get()).isSameAs(boom); + assertThat(downstream.completed.get()).isNull(); + } + + @Test + void exceptionalCompletionShortCircuitsWrap() { + // Sanity case: the throwable branch must NOT touch wrapFunction. + CapturingResultFuture downstream = new CapturingResultFuture(); + DelegatingAsyncTableResultFuture bridge = + new DelegatingAsyncTableResultFuture( + downstream, /* needsWrapping */ true, /* isInternalResultType */ false); + + RuntimeException boom = new RuntimeException("user boom"); + bridge.getCompletableFuture().completeExceptionally(boom); + + assertThat(downstream.error.get()).isSameAs(boom); + assertThat(downstream.completed.get()).isNull(); + } + + @Test + void identityWrappingForwardsCollectionUnchanged() { + // needsWrapping=false → wrapFunction is identity; the original collection must reach + // the delegate untouched. + CapturingResultFuture downstream = new CapturingResultFuture(); + DelegatingAsyncTableResultFuture bridge = + new DelegatingAsyncTableResultFuture( + downstream, /* needsWrapping */ false, /* isInternalResultType */ false); + + Collection outs = Collections.singletonList("payload"); + bridge.getCompletableFuture().complete(outs); + + assertThat(downstream.completed.get()).isSameAs(outs); + assertThat(downstream.error.get()).isNull(); + } + + /** + * Collection whose iterator throws on first call — used to force wrapExternal / wrapInternal to + * fail during {@code wrapFunction.apply(outs)} inside accept(). + */ + private static final class ThrowingOnIterateCollection extends AbstractCollection { + private final RuntimeException toThrow; + + ThrowingOnIterateCollection(RuntimeException toThrow) { + this.toThrow = toThrow; + } + + @Override + public Iterator iterator() { + throw toThrow; + } + + @Override + public int size() { + return 1; + } + } + + /** Minimal ResultFuture that records the last complete/completeExceptionally call. */ + private static final class CapturingResultFuture implements ResultFuture { + final AtomicReference> completed = new AtomicReference<>(); + final AtomicReference error = new AtomicReference<>(); + + @Override + public void complete(Collection result) { + completed.set(result); + } + + @Override + public void completeExceptionally(Throwable error) { + this.error.set(error); + } + + @Override + public void complete(CollectionSupplier supplier) { + throw new UnsupportedOperationException(); + } + } +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinRunnerTimeoutTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinRunnerTimeoutTest.java new file mode 100644 index 0000000000000..65bc3ca1419a7 --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinRunnerTimeoutTest.java @@ -0,0 +1,943 @@ +/* + * 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.join; + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.DefaultOpenContext; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.streaming.api.datastream.AsyncDataStream; +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.streaming.api.operators.async.AsyncWaitOperatorFactory; +import org.apache.flink.streaming.util.MockStreamingRuntimeContext; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.api.DataTypes; +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.runtime.collector.TableFunctionResultFuture; +import org.apache.flink.table.runtime.generated.GeneratedFunctionWrapper; +import org.apache.flink.table.runtime.generated.GeneratedResultFutureWrapper; +import org.apache.flink.table.runtime.operators.join.lookup.AsyncLookupJoinRunner; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.runtime.util.RowDataHarnessAssertor; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.Row; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.table.data.StringData.fromString; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.insertRecord; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the timeout handling in {@link AsyncLookupJoinRunner}. */ +public class AsyncLookupJoinRunnerTimeoutTest { + + private static final int ASYNC_BUFFER_CAPACITY = 100; + + private final TypeSerializer inSerializer = + new RowDataSerializer( + DataTypes.INT().getLogicalType(), DataTypes.STRING().getLogicalType()); + + private final RowDataHarnessAssertor assertor = + new RowDataHarnessAssertor( + new LogicalType[] { + DataTypes.INT().getLogicalType(), + DataTypes.STRING().getLogicalType(), + DataTypes.INT().getLogicalType(), + DataTypes.STRING().getLogicalType() + }); + + private final DataType rightRowDataType = + DataTypes.ROW( + DataTypes.FIELD("f0", DataTypes.INT()), + DataTypes.FIELD("f1", DataTypes.STRING())) + .bridgedTo(RowData.class); + + @SuppressWarnings({"unchecked", "rawtypes"}) + private final DataStructureConverter fetcherConverter = + (DataStructureConverter) DataStructureConverters.getConverter(rightRowDataType); + + private final RowDataSerializer rightRowSerializer = + (RowDataSerializer) + InternalSerializers.create(rightRowDataType.getLogicalType()); + + // --------------------------------------------------------------------------------- + // Unit-level tests (direct asyncInvoke/timeout calls) + // --------------------------------------------------------------------------------- + + @Test + public void testTimeoutPropagatesException() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner(new NeverCompletingFetcherFunction(), false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + joinRunner.timeout(input, resultFuture); + + assertThat(resultFuture.error).isInstanceOf(TimeoutException.class); + assertThat(resultFuture.result).isNull(); + + joinRunner.close(); + } + + @Test + public void testTimeoutWithFallbackResultsInnerJoin() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner(new FallbackOnTimeoutFetcherFunction(), false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + joinRunner.timeout(input, resultFuture); + + assertThat(resultFuture.error).isNull(); + assertThat(resultFuture.result).hasSize(1); + RowData joined = resultFuture.result.iterator().next(); + assertThat(joined.getInt(0)).isEqualTo(1); + assertThat(joined.getString(1).toString()).isEqualTo("a"); + assertThat(joined.getInt(2)).isEqualTo(1); + assertThat(joined.getString(3).toString()).isEqualTo("fallback"); + + joinRunner.close(); + } + + @Test + public void testTimeoutWithFallbackResultsLeftJoin() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner(new FallbackOnTimeoutFetcherFunction(), true); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + joinRunner.timeout(input, resultFuture); + + assertThat(resultFuture.error).isNull(); + assertThat(resultFuture.result).hasSize(1); + RowData joined = resultFuture.result.iterator().next(); + assertThat(joined.getInt(0)).isEqualTo(1); + assertThat(joined.getString(1).toString()).isEqualTo("a"); + assertThat(joined.getInt(2)).isEqualTo(1); + assertThat(joined.getString(3).toString()).isEqualTo("fallback"); + + joinRunner.close(); + } + + @Test + public void testTimeoutWithEmptyFallbackInnerJoin() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner(new EmptyFallbackOnTimeoutFetcherFunction(), false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + joinRunner.timeout(input, resultFuture); + + assertThat(resultFuture.error).isNull(); + assertThat(resultFuture.result).isEmpty(); + + joinRunner.close(); + } + + @Test + public void testTimeoutWithEmptyFallbackLeftJoin() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner(new EmptyFallbackOnTimeoutFetcherFunction(), true); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + joinRunner.timeout(input, resultFuture); + + assertThat(resultFuture.error).isNull(); + assertThat(resultFuture.result).hasSize(1); + RowData joined = resultFuture.result.iterator().next(); + assertThat(joined.getInt(0)).isEqualTo(1); + assertThat(joined.getString(1).toString()).isEqualTo("a"); + assertThat(joined.isNullAt(2)).isTrue(); + assertThat(joined.isNullAt(3)).isTrue(); + + joinRunner.close(); + } + + @Test + public void testTimeoutWhenFutureAlreadyCompleted() throws Exception { + AsyncLookupJoinRunner joinRunner = createRunner(new SyncCompletingFetcherFunction(), false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + + assertThat(resultFuture.result).hasSize(1); + RowData joined = resultFuture.result.iterator().next(); + assertThat(joined.getInt(2)).isEqualTo(1); + assertThat(joined.getString(3).toString()).isEqualTo("sync"); + + int futureCountBefore = joinRunner.getAllResultFutures().size(); + joinRunner.timeout(input, resultFuture); + int futureCountAfter = joinRunner.getAllResultFutures().size(); + + assertThat(futureCountAfter).isEqualTo(futureCountBefore); + assertThat(resultFuture.error).isNull(); + + joinRunner.close(); + } + + @Test + public void testTimeoutFetcherThrowsException() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner(new ThrowingTimeoutFetcherFunction(), false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + joinRunner.timeout(input, resultFuture); + + assertThat(resultFuture.error) + .isInstanceOf(RuntimeException.class) + .hasMessage("fetcher timeout failed"); + + joinRunner.close(); + } + + @Test + public void testTimeoutDiscardsOldFutureAndCreatesNew() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner(new FallbackOnTimeoutFetcherFunction(), false); + + int initialFutureCount = joinRunner.getAllResultFutures().size(); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + joinRunner.timeout(input, resultFuture); + + assertThat(joinRunner.getAllResultFutures()).hasSize(initialFutureCount); + + joinRunner.close(); + } + + @Test + public void testNonIdentityResultConverter() throws Exception { + DataType externalRightRowDataType = + DataTypes.ROW( + DataTypes.FIELD("f0", DataTypes.INT()), + DataTypes.FIELD("f1", DataTypes.STRING())); + @SuppressWarnings({"unchecked", "rawtypes"}) + DataStructureConverter externalConverter = + (DataStructureConverter) + DataStructureConverters.getConverter(externalRightRowDataType); + + AsyncLookupJoinRunner joinRunner = + createRunner( + new ExternalTypeSyncCompletingFetcher(), + externalConverter, + new AsyncLookupJoinHarnessTest.TestingFetcherResultFuture(), + false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + + assertThat(resultFuture.error).isNull(); + assertThat(resultFuture.result).hasSize(1); + RowData joined = resultFuture.result.iterator().next(); + assertThat(joined.getInt(0)).isEqualTo(1); + assertThat(joined.getString(1).toString()).isEqualTo("a"); + assertThat(joined.getInt(2)).isEqualTo(1); + assertThat(joined.getString(3).toString()).isEqualTo("converted"); + + joinRunner.close(); + } + + @Test + public void testFetcherCompletesExceptionally() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner(new SyncExceptionalFetcherFunction(), false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + + assertThat(resultFuture.result).isNull(); + assertThat(resultFuture.error) + .isInstanceOf(RuntimeException.class) + .hasMessage("async fetch failed"); + + joinRunner.close(); + } + + @Test + public void testLateCompleteIgnoredAfterTimeout() throws Exception { + AsyncLookupJoinRunner joinRunner = createRunner(new StoringFetcherFunction(), false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + ResultFuture innerFuture = StoringFetcherFunction.lastStoredFuture; + + joinRunner.timeout(input, resultFuture); + assertThat(resultFuture.error).isInstanceOf(TimeoutException.class); + + resultFuture.error = null; + resultFuture.result = null; + + innerFuture.complete(Collections.singletonList(GenericRowData.of(1, fromString("late")))); + + assertThat(resultFuture.result).isNull(); + assertThat(resultFuture.error).isNull(); + + joinRunner.close(); + } + + @Test + public void testLateCompleteExceptionallyIgnoredAfterTimeout() throws Exception { + AsyncLookupJoinRunner joinRunner = createRunner(new StoringFetcherFunction(), false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + ResultFuture innerFuture = StoringFetcherFunction.lastStoredFuture; + + joinRunner.timeout(input, resultFuture); + assertThat(resultFuture.error).isInstanceOf(TimeoutException.class); + + resultFuture.error = null; + resultFuture.result = null; + + innerFuture.completeExceptionally(new RuntimeException("late error")); + + assertThat(resultFuture.result).isNull(); + assertThat(resultFuture.error).isNull(); + + joinRunner.close(); + } + + @Test + public void testJoinConditionDelegateCompletesExceptionally() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner( + new SyncCompletingFetcherFunction(), + fetcherConverter, + new ExceptionallyCompletingResultFuture(), + false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + + assertThat(resultFuture.result).isNull(); + assertThat(resultFuture.error) + .isInstanceOf(RuntimeException.class) + .hasMessage("join condition failed"); + + joinRunner.close(); + } + + @Test + public void testJoinConditionThrowsException() throws Exception { + AsyncLookupJoinRunner joinRunner = + createRunner( + new SyncCompletingFetcherFunction(), + fetcherConverter, + new ThrowingConditionResultFuture(), + false); + + RowData input = row(1, "a"); + CapturingResultFuture resultFuture = new CapturingResultFuture(); + + joinRunner.asyncInvoke(input, resultFuture); + + assertThat(resultFuture.result).isNull(); + assertThat(resultFuture.error) + .isInstanceOf(RuntimeException.class) + .hasMessage("join condition threw"); + + joinRunner.close(); + } + + // --------------------------------------------------------------------------------- + // Integration-level tests (via OneInputStreamOperatorTestHarness) + // --------------------------------------------------------------------------------- + + @Test + public void testTimeoutInnerJoinWithFallback() throws Exception { + SlowFetcherFunction.latch = new CountDownLatch(1); + + OneInputStreamOperatorTestHarness testHarness = + createHarnessWithTimeout(false, 10L); + + testHarness.open(); + testHarness.setProcessingTime(0L); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.processElement(insertRecord(1, "a")); + testHarness.processElement(insertRecord(3, "c")); + testHarness.processElement(insertRecord(5, "e")); + } + + testHarness.setProcessingTime(11L); + SlowFetcherFunction.latch.countDown(); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.endInput(); + testHarness.close(); + } + + List expectedOutput = new ArrayList<>(); + expectedOutput.add(insertRecord(1, "a", 1, "Timeout_1")); + expectedOutput.add(insertRecord(3, "c", 3, "Timeout_3")); + + assertor.assertOutputEquals("output wrong.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testTimeoutLeftJoinWithFallback() throws Exception { + SlowFetcherFunction.latch = new CountDownLatch(1); + + OneInputStreamOperatorTestHarness testHarness = + createHarnessWithTimeout(true, 10L); + + testHarness.open(); + testHarness.setProcessingTime(0L); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.processElement(insertRecord(1, "a")); + testHarness.processElement(insertRecord(5, "e")); + } + + testHarness.setProcessingTime(11L); + SlowFetcherFunction.latch.countDown(); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.endInput(); + testHarness.close(); + } + + List expectedOutput = new ArrayList<>(); + expectedOutput.add(insertRecord(1, "a", 1, "Timeout_1")); + expectedOutput.add(insertRecord(5, "e", null, null)); + + assertor.assertOutputEquals("output wrong.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testMixedNormalAndTimeoutInnerJoin() throws Exception { + MixedSpeedFetcherFunction.latch = new CountDownLatch(1); + + OneInputStreamOperatorTestHarness testHarness = + createHarnessWithTimeout(new MixedSpeedFetcherFunction(), false, 10L); + + testHarness.open(); + testHarness.setProcessingTime(0L); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.processElement(insertRecord(1, "a")); + testHarness.processElement(insertRecord(3, "c")); + testHarness.processElement(insertRecord(4, "d")); + testHarness.processElement(insertRecord(5, "e")); + } + + testHarness.setProcessingTime(11L); + + MixedSpeedFetcherFunction.latch.countDown(); + Thread.sleep(50); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.endInput(); + testHarness.close(); + } + + List expectedOutput = new ArrayList<>(); + expectedOutput.add(insertRecord(1, "a", 1, "Julian")); + expectedOutput.add(insertRecord(3, "c", 3, "Timeout_3")); + expectedOutput.add(insertRecord(4, "d", 4, "Fabian")); + + assertor.assertOutputEquals("output wrong.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testMixedNormalAndTimeoutLeftJoin() throws Exception { + MixedSpeedFetcherFunction.latch = new CountDownLatch(1); + + OneInputStreamOperatorTestHarness testHarness = + createHarnessWithTimeout(new MixedSpeedFetcherFunction(), true, 10L); + + testHarness.open(); + testHarness.setProcessingTime(0L); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.processElement(insertRecord(1, "a")); + testHarness.processElement(insertRecord(3, "c")); + testHarness.processElement(insertRecord(4, "d")); + testHarness.processElement(insertRecord(5, "e")); + } + + testHarness.setProcessingTime(11L); + + MixedSpeedFetcherFunction.latch.countDown(); + Thread.sleep(50); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.endInput(); + testHarness.close(); + } + + List expectedOutput = new ArrayList<>(); + expectedOutput.add(insertRecord(1, "a", 1, "Julian")); + expectedOutput.add(insertRecord(3, "c", 3, "Timeout_3")); + expectedOutput.add(insertRecord(4, "d", 4, "Fabian")); + expectedOutput.add(insertRecord(5, "e", null, null)); + + assertor.assertOutputEquals("output wrong.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testTimeoutLateResultIgnored() throws Exception { + SlowFetcherFunction.latch = new CountDownLatch(1); + + OneInputStreamOperatorTestHarness testHarness = + createHarnessWithTimeout(false, 10L); + + testHarness.open(); + testHarness.setProcessingTime(0L); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.processElement(insertRecord(1, "a")); + testHarness.processElement(insertRecord(3, "c")); + } + + testHarness.setProcessingTime(11L); + + SlowFetcherFunction.latch.countDown(); + Thread.sleep(50); + + synchronized (testHarness.getCheckpointLock()) { + testHarness.endInput(); + testHarness.close(); + } + + List expectedOutput = new ArrayList<>(); + expectedOutput.add(insertRecord(1, "a", 1, "Timeout_1")); + expectedOutput.add(insertRecord(3, "c", 3, "Timeout_3")); + + assertor.assertOutputEquals("output wrong.", expectedOutput, testHarness.getOutput()); + } + + // --------------------------------------------------------------------------------- + // Helper methods + // --------------------------------------------------------------------------------- + + @SuppressWarnings({"unchecked", "rawtypes"}) + private AsyncLookupJoinRunner createRunner( + AsyncFunction fetcherFunction, boolean isLeftOuterJoin) throws Exception { + return createRunner( + fetcherFunction, + fetcherConverter, + new AsyncLookupJoinHarnessTest.TestingFetcherResultFuture(), + isLeftOuterJoin); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private AsyncLookupJoinRunner createRunner( + AsyncFunction fetcherFunction, + DataStructureConverter converter, + TableFunctionResultFuture resultFutureTemplate, + boolean isLeftOuterJoin) + throws Exception { + AsyncLookupJoinRunner joinRunner = + new AsyncLookupJoinRunner( + new GeneratedFunctionWrapper(fetcherFunction), + converter, + new GeneratedResultFutureWrapper<>(resultFutureTemplate), + new GeneratedFunctionWrapper( + new LookupJoinHarnessTest.TestingPreFilterCondition()), + rightRowSerializer, + isLeftOuterJoin, + ASYNC_BUFFER_CAPACITY); + joinRunner.setRuntimeContext(new MockStreamingRuntimeContext(1, 0)); + joinRunner.open(DefaultOpenContext.INSTANCE); + return joinRunner; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private OneInputStreamOperatorTestHarness createHarnessWithTimeout( + boolean isLeftJoin, long timeoutMs) throws Exception { + return createHarnessWithTimeout(new SlowFetcherFunction(), isLeftJoin, timeoutMs); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private OneInputStreamOperatorTestHarness createHarnessWithTimeout( + AsyncFunction fetcher, boolean isLeftJoin, long timeoutMs) + throws Exception { + RichAsyncFunction joinRunner = + new AsyncLookupJoinRunner( + new GeneratedFunctionWrapper(fetcher), + fetcherConverter, + new GeneratedResultFutureWrapper<>( + new AsyncLookupJoinHarnessTest.TestingFetcherResultFuture()), + new GeneratedFunctionWrapper( + new LookupJoinHarnessTest.TestingPreFilterCondition()), + rightRowSerializer, + isLeftJoin, + ASYNC_BUFFER_CAPACITY); + + return new OneInputStreamOperatorTestHarness<>( + new AsyncWaitOperatorFactory<>( + joinRunner, + timeoutMs, + ASYNC_BUFFER_CAPACITY, + AsyncDataStream.OutputMode.ORDERED), + inSerializer); + } + + // --------------------------------------------------------------------------------- + // Test utilities + // --------------------------------------------------------------------------------- + + private static final class CapturingResultFuture implements ResultFuture { + Collection result; + Throwable error; + + @Override + public void complete(Collection result) { + this.result = result; + } + + @Override + public void complete(CollectionSupplier supplier) { + try { + this.result = supplier.get(); + } catch (Exception e) { + this.error = e; + } + } + + @Override + public void completeExceptionally(Throwable error) { + this.error = error; + } + } + + // --------------------------------------------------------------------------------- + // Test fetcher functions + // --------------------------------------------------------------------------------- + + /** Never completes asyncInvoke; uses default timeout (TimeoutException). */ + public static final class NeverCompletingFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + private static final long serialVersionUID = 1L; + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) {} + } + + /** Never completes asyncInvoke; provides fallback results on timeout. */ + public static final class FallbackOnTimeoutFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + private static final long serialVersionUID = 1L; + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) {} + + @Override + public void timeout(RowData input, ResultFuture resultFuture) { + int id = input.getInt(0); + resultFuture.complete( + Collections.singletonList(GenericRowData.of(id, fromString("fallback")))); + } + } + + /** Never completes asyncInvoke; returns empty collection on timeout. */ + public static final class EmptyFallbackOnTimeoutFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + private static final long serialVersionUID = 1L; + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) {} + + @Override + public void timeout(RowData input, ResultFuture resultFuture) { + resultFuture.complete(Collections.emptyList()); + } + } + + /** Completes asyncInvoke synchronously. */ + public static final class SyncCompletingFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + private static final long serialVersionUID = 1L; + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) { + int id = input.getInt(0); + resultFuture.complete( + Collections.singletonList(GenericRowData.of(id, fromString("sync")))); + } + } + + /** Never completes asyncInvoke; throws exception on timeout. */ + public static final class ThrowingTimeoutFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + private static final long serialVersionUID = 1L; + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) {} + + @Override + public void timeout(RowData input, ResultFuture resultFuture) throws Exception { + throw new RuntimeException("fetcher timeout failed"); + } + } + + /** Completes asyncInvoke synchronously with external Row objects. */ + public static final class ExternalTypeSyncCompletingFetcher extends AbstractRichFunction + implements AsyncFunction { + private static final long serialVersionUID = 1L; + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) { + resultFuture.complete(Collections.singletonList(Row.of(input.getInt(0), "converted"))); + } + } + + /** Completes asyncInvoke synchronously with completeExceptionally. */ + public static final class SyncExceptionalFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + private static final long serialVersionUID = 1L; + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) { + resultFuture.completeExceptionally(new RuntimeException("async fetch failed")); + } + } + + /** Stores inner ResultFuture reference for late-completion tests. */ + public static final class StoringFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + private static final long serialVersionUID = 1L; + static volatile ResultFuture lastStoredFuture; + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) { + lastStoredFuture = resultFuture; + } + } + + /** + * A fetcher that blocks all async lookups on a latch, providing fallback data on timeout. Used + * to test the timeout handling in {@link AsyncLookupJoinRunner} via the operator harness. + */ + public static final class SlowFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + + private static final long serialVersionUID = 1L; + + private static final Map> normalData = new HashMap<>(); + private static final Map> fallbackData = new HashMap<>(); + + static volatile CountDownLatch latch; + + static { + normalData.put( + 1, Collections.singletonList(GenericRowData.of(1, fromString("Julian")))); + normalData.put( + 3, + Arrays.asList( + GenericRowData.of(3, fromString("Jark")), + GenericRowData.of(3, fromString("Jackson")))); + normalData.put( + 4, Collections.singletonList(GenericRowData.of(4, fromString("Fabian")))); + + fallbackData.put( + 1, Collections.singletonList(GenericRowData.of(1, fromString("Timeout_1")))); + fallbackData.put( + 3, Collections.singletonList(GenericRowData.of(3, fromString("Timeout_3")))); + fallbackData.put( + 4, Collections.singletonList(GenericRowData.of(4, fromString("Timeout_4")))); + } + + private transient ExecutorService executor; + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + this.executor = Executors.newFixedThreadPool(2); + } + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) { + int id = input.getInt(0); + executor.submit( + () -> { + try { + latch.await(); + } catch (InterruptedException e) { + return; + } + Collection result = normalData.get(id); + resultFuture.complete(result != null ? result : Collections.emptyList()); + }); + } + + @Override + public void timeout(RowData input, ResultFuture resultFuture) { + int id = input.getInt(0); + Collection result = fallbackData.get(id); + resultFuture.complete(result != null ? result : Collections.emptyList()); + } + + @Override + public void close() throws Exception { + super.close(); + if (executor != null && !executor.isShutdown()) { + executor.shutdownNow(); + } + } + } + + /** + * A fetcher with mixed behavior: some keys complete immediately, others block on a latch. On + * timeout, returns fallback data for key 3 and empty for others. + */ + public static final class MixedSpeedFetcherFunction extends AbstractRichFunction + implements AsyncFunction { + + private static final long serialVersionUID = 1L; + + private static final Map> fastData = new HashMap<>(); + private static final Map> slowData = new HashMap<>(); + private static final Map> fallbackData = new HashMap<>(); + + static volatile CountDownLatch latch; + + static { + fastData.put(1, Collections.singletonList(GenericRowData.of(1, fromString("Julian")))); + fastData.put(4, Collections.singletonList(GenericRowData.of(4, fromString("Fabian")))); + + slowData.put( + 3, + Arrays.asList( + GenericRowData.of(3, fromString("Jark")), + GenericRowData.of(3, fromString("Jackson")))); + slowData.put(5, Collections.singletonList(GenericRowData.of(5, fromString("Slow_5")))); + + fallbackData.put( + 3, Collections.singletonList(GenericRowData.of(3, fromString("Timeout_3")))); + } + + private transient ExecutorService executor; + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + this.executor = Executors.newFixedThreadPool(2); + } + + @Override + public void asyncInvoke(RowData input, ResultFuture resultFuture) { + int id = input.getInt(0); + List fast = fastData.get(id); + if (fast != null) { + resultFuture.complete(fast); + return; + } + executor.submit( + () -> { + try { + latch.await(); + } catch (InterruptedException e) { + return; + } + Collection result = slowData.get(id); + resultFuture.complete(result != null ? result : Collections.emptyList()); + }); + } + + @Override + public void timeout(RowData input, ResultFuture resultFuture) { + int id = input.getInt(0); + Collection result = fallbackData.get(id); + resultFuture.complete(result != null ? result : Collections.emptyList()); + } + + @Override + public void close() throws Exception { + super.close(); + if (executor != null && !executor.isShutdown()) { + executor.shutdownNow(); + } + } + } + + /** Calls completeExceptionally on the delegate in complete(). */ + public static final class ExceptionallyCompletingResultFuture + extends TableFunctionResultFuture { + private static final long serialVersionUID = 1L; + + @Override + @SuppressWarnings("unchecked") + public void complete(Collection result) { + getResultFuture().completeExceptionally(new RuntimeException("join condition failed")); + } + } + + /** Throws exception in complete(). */ + public static final class ThrowingConditionResultFuture + extends TableFunctionResultFuture { + private static final long serialVersionUID = 1L; + + @Override + public void complete(Collection result) { + throw new RuntimeException("join condition threw"); + } + } +}