Motivation
A common UDF type is the ScalarFunction. This works well for CPU-intensive operations, but less well for IO bound or otherwise long-running computations. One example of this is remote calls to external systems where networking, serialization, and database lookups might dominate the execution time. StreamTask has a single thread serially executing operators and their contained calls, which happen synchronously in the remote call case. Since each call can take, say, 1 second or more, that limits throughput and the overall performance, potentially accumulating backpressure to the upstream operator. The solution is to either: increase the parallelism of the query (resulting in a higher resource cost, overhead, etc.) or asynchronously fire off many requests concurrently and receive results as they complete. This FLIP aims to address the latter solution by introducing AsyncScalarFunction, a new UDF type which allows for issuing concurrent function calls.
Scope
There are lots of combinations of modes and Job types in Flink such as the changelog mode and streaming vs batch. To make clear the scope this FLIP intends to cover, the functionality will be limited to the following:
- Ordered Async operators: Much discussion has been centered around which changelog modes, SQL queries could be compatible with an operator which allowed unordered results, since there is a performance benefit. For now we'll only consider an operator that retains the input ordering.
- Streaming mode: Some of the design considerations we're considering are focused on streaming. To get good performance on batch, it's possible we might want to allow batching of async calls, but we're not addressing this at the moment.
Public Interfaces
The primary public class is AsyncScalarFunction, for being the base class of all async scalar functions.
The type is parameterized with a return type for the eval call. This is similar to the definition of AsyncTableFunction.
public class AsyncScalarFunction extends UserDefinedFunction { @Override public final FunctionKind getKind() { return FunctionKind.ASYNC_SCALAR; } @Override public TypeInference getTypeInference(DataTypeFactory typeFactory) { TypeInference val = TypeInferenceExtractor.forAsyncScalarFunction(typeFactory, getClass()); return val; } }
An example implementing class could be the following:
public class RemoteCallFunction extends AsyncScalarFunction { private ExternalClient client; private ExecutorService executor; public RemoteCallFunction() { } @Override public void open(FunctionContext context) throws Exception { client = new Client(); executor = Executors.newFixedThreadPool( context.getJobParameter("in-flight-requests", 10)); } @Override public void close() throws Exception { client.close(); executor.shutdownNow(); } public final void eval( CompletableFuture<String> future, String param1, int param2) { executor.submit(() -> { try { String resp = client.rpc(param1, param2); future.complete(resp); } catch (Throwable t) { future.completeExceptionally(t); } }); } }
As with the standard ScalarFunction, there is an eval method, but with a 0th parameter of the type CompletableFuture<String> future. This is the primary method used to invoke the async functionality. The generic parameter of the future is used to infer the return type for the type system.
New configurations will be introduced for the functionality, similar in nature to table.exec.async-lookup.*
:
table.exec.async-scalar.buffer-capacity: 10 table.exec.async-scalar.timeout: 30s table.exec.async-scalar.retry-strategy: FIXED_DELAY table.exec.async-scalar.fixed-delay: 10s table.exec.async-scalar.max-attempts: 3
These options ideally would be function scoped, but since `ConfigOption` doesn't make it easy to have a per-function config, they are global. Future work could allow these to be overridden on a per definition basis.
The following configurations will be available:
Name (Prefix table.exec.async-scalar) | Meaning |
---|---|
buffer-capacity | The number of outstanding requests the operator allows at once |
timeout | The total time which can pass before the invocation (including retries) is considered timed out and task execution is failed |
retry-strategy | FIXED_DELAY is for a retry after a fixed amount of time |
retry-delay | The time to wait between retries for the FIXED_DELAY strategy. Could be the base delay time for a (not yet proposed) exponential backoff. |
max-attempts | The maximum number of attempts while retrying. |
Proposed Changes
Planner Changes
Split Rules
One of the areas that have been used as inspiration for planner changes are the python calc rules. Most of the split rules (rules for complex calc nodes being split into multiple simpler calc nodes) will be generalized and
shared between the two, since remote python calls and async calls more generally share much of the same structure. If done correctly, the intention is to simplify the async operator to handle only FlinkLogicalCalcs
which contain async UDF calls in projections and no other calc logic (non async calls, field accesses, conditions).
The high level motivation is that anything that comes after an async call is easier to chain as a series of operators rather than internally within a single operator.
Specifically, PythonCalcSplitRuleBase will be generalized into RemoteCalcSplitRuleBase. It will be parameterized with a RemoteCalcCallFinder which can be used to analyze the RexNode
s to look for python or async calls.
public interface RemoteCalcCallFinder { // This RexNode contains either directly or indirectly a remote call // of the specified type. boolean containsRemoteCall(RexNode node); // This RexNode contains either directly or indirectly a call which is not // the specified remote type. boolean containsNonRemoteCall(RexNode node); // This RexNode is a remote call of the specified type. boolean isRemoteCall(RexNode node); // This RexNode is a call that is not the specified type. boolean isNonRemoteCall(RexNode node); }
This will allow for PythonCalcCallFinder and AsyncCalcCallFinder implementations.
The rules we intend to adopt split up a FlinkLogicalCalc into two (or more ultimately) FlinkLogicalCalcs which feed into one another. The async split rules shared with Python will be:
Rule | Original RelNode | Becomes (Bottom ==> Top) |
---|---|---|
SPLIT_CONDITION Splits FlinkLogicalCalcs which contain Remote functions in the condition into |
|
==>
|
SPLIT_PROJECT Splits projections with async functions and non async |
|
==>
|
SPLIT_PROJECTION_REX_FIELD Splits field accesses from the result of an async call in projections |
|
==>
|
SPLIT_CONDITION_REX_FIELD Splits field accesses from the result of an async call in condition |
|
==>
|
EXPAND_PROJECT Splits field accesses as inputs to async calls into two FlinkLogicalCalcs. |
|
==>
|
PUSH_CONDITION Pushes conditions down to minimize rows requiring the async call, |
|
==>
|
Async Specific: NESTED_SPLIT If there is a call with an async call as an argument, then it needs to be split |
|
==>
|
Async Specific: ONE_ASYNC_PROJECTION_PER_CALC If there are multiple projections containing async calls, it splits them into two |
|
==>
|
Physical Rules
In additional the split rules, there will also need to be a PhysicalAsyncCalcRule which converts FlinkLogicalCalcs to PhysicalAsyncCalcs.
This will check for the existence of any async calls in the calc, using the same AsyncCalcCallFinder logic above.
Disallowing Async functionality when not supported
It is most prudent to only allow async behavior where it is known to not violate SQL semantics. To do this, rules will be introduced which contain query structures which we don’t want to allow and if found, all of the async calls will be executed in synchronous mode.
This can be done by introducing a new trait AsyncOperatorModeTrait, which comes in sync mode and async mode (default), and which will be attached to a FlinkLogicalCalc if it contains async calls which we would prefer to execute in sync mode. Execution in synchronous mode just utilizes the same stack of as async, but waits on the result immediately after issuing the request.
An example of a query which could have unintended results without explicit handling:
- Queries with ordering semantics:
- e.g. SELECT func(f1) FROM Table ORDER BY f2;
Here, we expect that the results of the query will be ordered by f2. We wouldn't want to return results in the completion order from async function func.
We can solve it by either outputting in ORDERED, and ensuring that we return the results in the input order, or by putting it into synchronous mode and ensuring ordering by doing one at a time.
- e.g. SELECT func(f1) FROM Table ORDER BY f2;
- Others? Would be great to get feedback on other cases that should be considered.
For the first version of this functionality where the operator outputs only in ordered mode, synchronous mode may not need to be enabled.
Runtime Changes
Code Generation
Current code generation in Flink for ScalarFunctions assume that one call can be synchronously fed into another and results can be set on the output record right after being issued.
Neither of these will hold once we have async support. There are two phases for generated async code:
Call issuing phase
Same as a normal sync invocation w.r.t. to converting all of the parameters and calling the UDF, but with the result being returned with a
Future
.
Result collection phase
Must wait for all async calls issued and also convert result types, if appropriate.
Only once all results are ready can an output record be created and set.
Must happen with a callback on the
Future
s rather than synchronously.
Note that how the operator is going to invoke the generated code has to do with what planner rules are in effect above. If every Async Operator is guaranteed to have only parallel async calls and no other generated Java/sql operations, then the generated code can be simplified, leaving support for everything else to existing Flink operators. This is a similar approach to that taken by Python. With the last split rule above, the code can be simplified further by requiring only one async request per operator.
Since the code generator already supports generating AsyncFunctions (currently used by lookup joins), this will be used with the main logic in asyncInvoke. The body of that method will use existing code generation to call the UDF and do the appropriate casting for the various arguments. Additional logic will capture the UDF result Future, set a callback, convert results, and complete the output row.
Utilizing a class AsyncDelegatingResultFuture similar to the existing DelegatingResultFuture (used for lookup joins), the generated method could look similar to the following:
@Override public void asyncInvoke(RowData input, ResultFuture<RowData> resultFuture) throws Exception { // Invokes callbacks on resultFuture once the async call is complete. final AsyncDelegatingResultFuture delegatingFuture = AsyncDelegatingResultFuture(resultFuture); try { Function<Object, GenericRowData> outputFactory = new Function<Object, GenericRowData>() { @Override public GenericRowData apply(Object udfResult) { // Gather the results and return the output object final GenericRowData out = new GenericRowData(2); out.setField(0, delegatingFuture.getSynchronousResult(0)); out.setField(1, udfResult); return out; } }; // Once it sees that the async future is done, the factory will be used to get the resulting output row delegatingFuture.setOutputFactory(outputFactory); // If an input is needed in the next operator, pass it along int passThroughField = input.getInt(0); delegatingFuture.addSynchronousResult(passThroughField); // Create a new future object and invoke the UDF. // The result will be converted to the internal type before calling the output factory. CompletableFuture<?> udfResultFuture = delegatingFuture.createAsyncFuture(typeConverter); asyncScalarFunctionUdf.eval(udfResultFuture); } catch (Throwable e) { resultFuture.completeExceptionally(e); } }
Operator
Since the call to the AsyncScalarFunction is wrapped in a AsyncFunction taking input rows, we have the benefit of using the existing class AsyncWaitOperator, which handles ordering, checkpointing, timeouts and other implementation details. Since only ordered results are handled in this scope, ORDERED will be the default behavior.
The PhysicalAsyncCalcs
mentioned in the planning phase will translate to an exec node, which creates the transformation containing this operator.
Compatibility, Deprecation, and Migration Plan
This is only introducing new code paths, namely the use of AsyncScalarFunction, so there should be no compatibility issues with existing jobs/SQL queries.
Test Plan
Unit tests on all of the components
ITCases that cover:
Each split rule
Various query structures which include fallback cases
The ability to verify which plan was utilized, async or sync
Rejected Alternatives
AsyncTableFunction using a lookup Join
This requires you to model the lookups as a join with a table. For example:
create TEMPORARY TABLE RemoteTable(table_lookup_key string, resp string, PRIMARY KEY (table_lookup_key) NOT ENFORCED) with ('connector' = 'remote_call'); SELECT i.table_lookup_key, resp FROM Inputs as i JOIN RemoteTable r FOR SYSTEM_TIME AS OF i.proc_time as a ON i.table_lookup_key = r.table_lookup_key;
+ Already implemented
+ High performance
+ Good for modeling external databases with a table interface
- Can’t easily invoke the lookup multiple times per joining row
- Requires proc_time time attribute, which may be unnecessary or not already exist on a pre defined table
- Unintuitive compared to a scalar function
Polymorphic table function
This already exists in some capacity in Flink with window functions.
This would allow you to effectively specify a number of input keys for some remote call and issue calls at a high volume. For example:
SELECT * FROM TABLE (REMOTE_CALL (Input => Table(TableToLookup) as d, Col => DESCRIPTOR("table_lookup_key")));
+ More intuitive than a lookup join
- No support for user defined functions with PTFs.