...
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
To limit the scope for consideration, the goal is to allow AsyncScalarFunctions to be used in:
Projections e.g. SELECT func(f1) FROM ...
Simple Conditions e.g. SELECT ... FROM T1 WHERE func(f1) > 10
Out of consideration:
Join Conditions e.g. SELECT ... T1 INNER JOIN T1 ON func(f1) = func(f2)
Subquery Conditions e.g. SELECT ... WHERE func(f1) IN (SELECT id FROM Table)
Aggregate Projections e.g. SELECT f1, func(count(*)) from t1 group by f1
Other places where non deterministic ordering or other considerations may complicate execution or break SQL semantics.
Regarding the changelog modes supported by the async operator, incoming row types to support are:
Insert (+I)
Update After (+U)
Both of these types can be passed on to the UDF and executed. Any other row types won’t be supported.
...
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.
Code Block |
---|
public class AsyncScalarFunction<T> 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:
Code Block |
---|
public class RemoteCallFunction extends AsyncScalarFunction<String> {
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(); |
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.
Code Block |
---|
public class AsyncScalarFunction<T> extends UserDefinedFunction { @Override public final FunctionKind getKind() { return FunctionKind.ASYNC_SCALARexecutor.shutdownNow(); } public final void @Override eval( public TypeInference getTypeInference(DataTypeFactory typeFactory)CompletableFuture<String> {future, TypeInference val = TypeInferenceExtractor.forAsyncScalarFunction(typeFactory, getClass()); String param1, return val; int } } |
An example implementing class could be the following:
Code Block |
---|
public class RemoteCallFunction extends AsyncScalarFunction<String> { private ExternalClient client;param2) { executor.submit(() -> { private ExecutorService executor; public RemoteCallFunction()try { } @Override publicString resp void= open(FunctionContext context) throws Exception { client.rpc(param1, param2); client = new Clientfuture.complete(resp); executor = Executors.newFixedThreadPool( } catch (Throwable t) { context.getJobParameter("in-flight-requests", 10)); } @Override public void close() throws Exception { future.completeExceptionally(t); 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 with a 0th parameter of the type CompletableFuture<String> future.
This is the primary method used to invoke the async functionality.
...
} |
As with the standard ScalarFunction, there is an eval method with a 0th parameter of the type CompletableFuture<String> future.
This is the primary method used to invoke the async functionality.
New configurations will be introduced for the functionality, similar in nature to table.exec.async-lookup.*
:
Code Block |
---|
table.exec.async-scalar.catalog.db.func-name.buffer-capacity: 10
table.exec.async-scalar.catalog.db.func-name.timeout: 30s
table.exec.async-scalar.catalog.db.func-name.output-mode: ORDERED
table.exec.async-scalar.catalog.db.func-name.retry-strategy: FIXED_DELAY
table.exec.async-scalar.catalog.db.func-name.fixed-delay: 10s
table.exec.async-scalar.catalog.db.func-name.max-attempts: 3
table.exec.async-scalar.system.func-name.buffer-capacity: 10 |
These options are scoped with the catalog, db, and function name as registered in the table environment so that any given definition can be configured. Similarly, a system function can be configured with the special prefix system, as in the last example.
The options have the following meanings:
...
Name (Prefix |
---|
table.exec.async-scalar.catalog.db.func-name |
---|
...
Name (Prefix table.exec.async-scalar.catalog.db.func-name) | Meaning |
---|---|
buffer-capacity | The number of outstanding requests the operator allows at once |
timeout | The time which can pass before a restart strategy is triggered |
output-mode | Depending on whether the planner deems it possible to allow for the more performant unordered option. |
retry-strategy | FIXED_DELAY is for a retry after a fixed amount of time |
fixed-delay | The time to wait between retries for the FIXED_DELAY strategy. |
max-attempts | The maximum number of attempts while retrying. |
...
Planner Changes
...
) | Meaning |
---|---|
buffer-capacity | The number of outstanding requests the operator allows at once |
timeout | The time which can pass before a restart strategy is triggered |
output-mode | Depending on whether the planner deems it possible to allow for the more performant unordered option. |
retry-strategy | FIXED_DELAY is for a retry after a fixed amount of time |
fixed-delay | The time to wait between retries for the FIXED_DELAY strategy. |
max-attempts | The maximum number of attempts while retrying. |
Proposed Changes
Planner Changes
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.
Code Block |
---|
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. |
|
|
Specifically, PythonCalcSplitRuleBase will be generalized into RemoteCalcSplitRuleBase. It will be parameterized with a RemoteCalcCallFinder which can be used to analyze the RexNodes to look for python or async calls.
Code Block |
---|
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);
} |
...
Rule | Original RelNode | Becomes (Bottom ==> Top) | |
---|---|---|---|
SPLIT_CONDITION Splits FlinkLogicalCalcs which contain Remote functions in the condition into |
|
==>
| |
SPLITPUSH_PROJECTCONDITION 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 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 | FlinkLogicalCalc
Projections:
Condition:
Async Specific: ONE_ASYNC_PROJECTION_PER_CALC If there are multiple projections containing async calls, it splits them into two |
| ||
|
| ||
==>
| |||
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 |
==>
|
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 utilizing output-mode as 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
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
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
Note that how the operator is going to invoke the generated code has to do with what rules are in effect above.
If every Async Operator is guaranteed to only invoke 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 the same approach taken by Python.
...
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 utilizing output-mode as 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.
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), it will be used with the fetching logic in the method asyncInvoke(RowData input, ResultFuture<Collection<RowData> result). 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 AsyncFunction ResultFuture.
Utilizing a class similar to the existing DelegatingResultFuture, the generated method could look similar to the following:
Code Block |
---|
@Override
public void asyncInvoke(RowData input, ResultFuture<RowData> resultFuture) throws Exception {
final AsyncDelegatingResultFuture delegatingFuture = AsyncDelegatingResultFuture(resultFuture);
try {
java.util.function.Function<Object, GenericRowData> outputFactory = new java.util.function.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, f.getSynchronousResult(0));
out.setField(1, udfResult);
return out;
}
};
// Once it sees that the 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);
udfInstance.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.
...