[Apache Flink] Mail FeedConfluence Syndication Feedhttps://cwiki.apache.org/confluenceFLIP-437: Support ML Models in Flink SQLTimo Walthertag:cwiki.apache.org,2009:page-296290787-62024-03-28T16:35:25Z2024-03-12T14:53:14Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~twalthr
">Timo Walther</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<h1 id="FLIP437:SupportMLModelsinFlinkSQL-Status">Status</h1><p><strong>Current state</strong>: Under Discussion</p><p><strong>Discussion thread</strong>: <a class="external-link" href="https://lists.apache.org/thread/9z94m2bv4w265xb5l2mrnh4lf9m28ccn" rel="nofollow">https://lists.apache.org/thread/9z94m2bv4w265xb5l2mrnh4lf9m28ccn</a><em><br/></em></p><p><strong>JIRA</strong>: <em>here (<- link to <a class="external-link" href="https://issues.apache.org/jira/browse/FLINK-XXXX" rel="nofollow">https://issues.apache.org/jira/browse/FLINK-XXXX</a>)<br/></em></p><p><strong>Released: </strong><Flink Version></p><p>Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).</p><h1 id="FLIP437:SupportMLModelsinFlinkSQL-Motivation">Motivation</h1><p>ML developers spend significant time on data cleaning, preprocessing, ingestion for ML training and inference with two sets of frameworks (e.g., Spark, Flink for data tasks, Tensorflow, PyTorch for ML tasks). Usually these frameworks are deployed in separate platforms, meaning developers have to rely on external orchestration systems and storage to stitch them into a cohesive workflow. Separating data processing tasks from the ML tasks also adds complexity to change management, data governance and lineage tracking etc. The rapid evolution of AI and GenAI is significantly influencing the data industry, steering it towards a unified streaming data platform architecture for almost all market players. In fact, ML is essentially another way of extracting insights from data, which logically is no different from the traditional data processing & analytics, but with more intensive computation requirements. Ideally there should be an unified set of APIs to describe the data processing and ML tasks for a more cohesive user experience. As the declarative APIs (SQL) is the common tongue for data processing and analytics, the natural evolution should be to add SQL support for ML tasks.</p><h1 id="FLIP437:SupportMLModelsinFlinkSQL-PublicInterfaces">Public Interfaces</h1><p>Public interfaces changes include new SQL syntax changes proposed below for model operations as well as new catalog model and catalog changes to operate on models.</p><h2 id="FLIP437:SupportMLModelsinFlinkSQL-CatalogModel(New)">Catalog Model (New)</h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">/** Interface for a model in a catalog. */
@PublicEvolving
public interface CatalogModel {
/**
* Get the unresolved input schema of the model.
*
* @return unresolved input schema of the model.
*/
Schema getInputSchema();
/**
* Get the unresolved output schema of the model.
*
* @return unresolved output schema of the model.
*/
Schema getOutputSchema();
/**
* Get comment of the model.
*
* @return comment of the model.
*/
String getComment();
/**
* Get a deep copy of the CatalogModel instance.
*
* @return a copy of the CatalogModel instance
*/
CatalogModel copy();
/**
* Copy the input model options into the CatalogModel instance.
*
* @return a copy of the CatalogModel instance with new model options.
*/
CatalogModel copy(Map<String, String> modelOptions);
}</pre>
</div></div><h2 id="FLIP437:SupportMLModelsinFlinkSQL-ResolvedCatalogModel(New)">Resolved Catalog Model (New)</h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@PublicEvolving
public interface ResolvedCatalogModel extends CatalogModel {
/**
* Returns the original, unresolved metadata object from the {@link Catalog}.
*
* <p>This method might be useful if catalog-specific object instances should be directly
* forwarded from the catalog to a factory.
*/
CatalogModel getOrigin();
/** Returns a fully resolved and validated {@link ResolvedSchema} inputSchema. */
ResolvedSchema getResolvedInputSchema();
/** Returns a fully resolved and validated {@link ResolvedSchema} outputSchema. */
ResolvedSchema getResolvedOutputSchema();
/**
* Serializes this instance into a map of string-based properties.
*
* <p>Compared to the pure table options in {@link #getModelOptions()}, the map includes input
* schema, output schema, kind, task, comment and options.
*/
Map<String, String> toProperties();
/**
* Creates an instance of {@link CatalogModel} from a map of string properties that were
* previously created with {@link ResolvedCatalogModel#toProperties()}.
*
* @param properties serialized version of a {@link ResolvedCatalogModel} that includes input
* schema, output schema, kind, task, comment and options.
*/
static CatalogModel fromProperties(Map<String, String> properties) {
return CatalogPropertiesUtil.deserializeCatalogModel(properties);
}
}</pre>
</div></div><h2 id="FLIP437:SupportMLModelsinFlinkSQL-Catalog(Change)">Catalog (Change)</h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">public interface Catalog {
/**
* Get names of all tables models under this database. An empty list is returned if none exists.
*
* @return a list of the names of all models in this database
* @throws DatabaseNotExistException if the database does not exist
* @throws CatalogException in case of any runtime exception
*/
default List<String> listModels(String databaseName)
throws DatabaseNotExistException, CatalogException {
throw new UnsupportedOperationException(
String.format("listModel(String) is not implemented for %s.", this.getClass()));
}
/**
* Returns a {@link CatalogModel} identified by the given {@link ObjectPath}.
*
* @param modelPath Path of the model
* @return The requested model
* @throws ModelNotExistException if the target does not exist
* @throws CatalogException in case of any runtime exception
*/
default CatalogModel getModel(ObjectPath modelPath)
throws ModelNotExistException, CatalogException {
throw new UnsupportedOperationException(
String.format("getModel(ObjectPath) is not implemented for %s.", this.getClass()));
}
/**
* Check if a model exists in this catalog.
*
* @param modelPath Path of the model
* @return true if the given model exists in the catalog false otherwise
* @throws CatalogException in case of any runtime exception
*/
default boolean modelExists(ObjectPath modelPath) throws CatalogException {
throw new UnsupportedOperationException(
String.format(
"modelExists(ObjectPath) is not implemented for %s.", this.getClass()));
}
/**
* Drop a model.
*
* @param modelPath Path of the model to be dropped
* @param ignoreIfNotExists Flag to specify behavior when the model does not exist: if set to
* false, throw an exception, if set to true, do nothing.
* @throws ModelNotExistException if the model does not exist
* @throws CatalogException in case of any runtime exception
*/
default void dropModel(ObjectPath modelPath, boolean ignoreIfNotExists)
throws ModelNotExistException, CatalogException {
throw new UnsupportedOperationException(
String.format(
"dropModel(ObjectPath, boolean) is not implemented for %s.",
this.getClass()));
}
/**
* Rename an existing model.
*
* @param modelPath Path of the model to be renamed
* @param newModelName the new name of the model
* @param ignoreIfNotExists Flag to specify behavior when the model does not exist: if set to
* false, throw an exception, if set to true, do nothing.
* @throws ModelNotExistException if the model does not exist
* @throws CatalogException in case of any runtime exception
*/
default void renameModel(ObjectPath modelPath, String newModelName, boolean ignoreIfNotExists)
throws ModelNotExistException, ModelAlreadyExistException, CatalogException {
throw new UnsupportedOperationException(
String.format(
"renameModel(ObjectPath, String, boolean) is not implemented for %s.",
this.getClass()));
}
/**
* Creates a new model.
*
* @param modelPath path of the model to be created
* @param model the CatalogModel definition
* @param ignoreIfExists flag to specify behavior when a model already exists at the given path:
* if set to false, it throws a ModelAlreadyExistException, if set to true, do nothing.
* @throws ModelAlreadyExistException if model already exists and ignoreIfExists is false
* @throws DatabaseNotExistException if the database in tablePath doesn't exist
* @throws CatalogException in case of any runtime exception
*/
default void createModel(ObjectPath modelPath, CatalogModel model, boolean ignoreIfExists)
throws ModelAlreadyExistException, DatabaseNotExistException, CatalogException {
throw new UnsupportedOperationException(
String.format(
"createModel(ObjectPath, CatalogModel, boolean) is not implemented for %s.",
this.getClass()));
}
/**
* Modifies an existing model. Note that the new and old {@link CatalogModel} must be of the
* same kind. For example, this doesn't allow altering a remote model to import model or native
* model, and vice versa.
*
* @param modelPath path of the model to be modified
* @param newModel the new CatalogModel definition
* @param ignoreIfNotExists flag to specify behavior when the model does not exist: if set to
* false, throw an exception, if set to true, do nothing.
* @throws ModelNotExistException if the model does not exist
* @throws CatalogException in case of any runtime exception
*/
default void alterModel(ObjectPath modelPath, CatalogModel newModel, boolean ignoreIfNotExists)
throws ModelNotExistException, CatalogException {
throw new UnsupportedOperationException(
String.format(
"alterModel(ObjectPath, CatalogModel, boolean) is not implemented for %s.",
this.getClass()));
}
}</pre>
</div></div><p><br/></p><p><br/></p><p><span style="font-size: 24.0px;letter-spacing: -0.01em;">Proposed Changes</span></p><h2 id="FLIP437:SupportMLModelsinFlinkSQL-ModelResource">Model Resource</h2><p>We propose to introduce ML Models as 1st class citizens into Flink SQL and the corresponding catalog, similar to Table, following the same resource hierarchy (catalog / database / model). A model resource can be described with CatalogModel class with the following properties:</p><ul class="ak-ul"><li><p>ModelKind (Enum): internal or external model</p></li><li><p>ModelTask (Enum): regression, classification, clustering, generation, etc.</p></li><li><p>ModelOptions (Map<String, String>): string-based model specific options, including algorithm hyper parameters, inference runtime parameters, etc. Mandatory options include </p><ul class="ak-ul" style="margin-left: 0.0px;"><li><p>‘Task’: which describes the kind of tasks the model is doing such as classification, text generation etc.</p></li><li><p>‘Provider’: which describe the provider for external models such as openai etc.</p></li></ul></li><li><p>InputSchema (Schema): expected model input column schemas.</p></li><li><p>OutputSchema (Schema): expected model output column schemas.</p></li><li><p>Comment (String): comment of the model</p></li></ul><p>A CatalogModel object can be resolved into a ResolvedCatalogModel object with resolved input and output schemas. A serializer (#toProperties) and deserializer (#fromProperties) should also be implemented for ResolvedCatalogModel for Flink Jobs with model invoking. </p><h2 id="FLIP437:SupportMLModelsinFlinkSQL-SQLSyntax">SQL Syntax</h2><p>We also propose to support the corresponding DDL and other Syntax in Flink SQL for Model lifecycle management (create / update / delete / get / list) as follows </p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default"># Create Model DDL Syntax
CREATE MODEL [IF NOT EXISTS] [catalog_name.][db_name.]model_name
[INPUT (input_column_list)]
[OUTPUT (outpt_column_list)]
[COMMENT model_comment]
WITH(model_option_list)
[AS query_statement]
#Example I: import a model from customer storage.
CREATE MODEL `my_import_model`
INPUT (f1 INT, f2 STRING)
OUTPUT (label FLOAT)
WITH(
'task' = 'regression',
'type' = 'import',
'format' = 'ONNX',
'ONNX.path' = 'http://storage.googleapis.com/storage/t.onnx',
)
#Example II: reference a remote model from customer endpoint.
CREATE MODEL `my_remote_model`
INPUT (f1 INT, f2 STRING)
OUTPUT (label STRING, probs ARRAY<FLOAT>)
WITH(
'task' = 'classification',
'type' = 'remote',
'provider' = 'openai',
'openai.endpoint' = 'https://api.openai.com/v1/llm/v1/chat',
'openai.api_key' = 'abcdefg'
)
#Example III: train a kmeans clustering model in Flink.
CREATE MODEL `my_native_model`
WITH(
'task' = 'clustering',
'type' = 'training',
'algorithm' = 'kmeans',
'kmeans.num_clusters' = '3'
) AS SELECT F1, F2 FROM `my_data`</pre>
</div></div><ul class="ak-ul"><li><p>INPUT and OUTPUT list to specify the model signature (input and output schema) if it is not self described.</p></li><li><p>CREATE TEMPORARY MODEL is also supported.</p></li><li><p>Model_option_list is a key-value pair list to specify the model training & inference and other information, including model task, algorithm, training & tuning parameters, runtime optimization parameters and resource management options (version, tag, label, description) etc..</p></li><li><p>The AS query_statement is used to provide the training data.</p></li></ul><p><br/></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default"># Alter Model DDL Syntax
# rename model syntax
ALTER MODEL [IF EXISTS][catalog_name.][database_name.]model_name
RENAME TO [catalog_name.][database_name.]new_model_name
# Alter model options syntax
ALTER MODEL [IF EXISTS] [catalog_name.][database_name.]model_name
SET (key1=val1[, key2=val2]...)
# Example I: rename model
ALTER MODEL `my_model` RENAME TO `my_new_model`
ALTER MODEL IF EXISTS `my_model` RENAME TO `my_new_model`
# Example II: alter model options
ALTER MODEL `my_model` SET (
tag = 'prod',
description = "new_description"
)</pre>
</div></div><ul class="ak-ul"><li><p>Alter model options only apply to model metadata but not model data.</p></li><li><p>If IF EXISTS is provided and the model doesn’t exist, nothing happens.</p></li><li><p>if IF EXISTS is provided and the model version doesn’t exist, nothing happens.</p></li></ul><p><br/></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default"># Drop Model DDL Syntax
DROP MODEL [IF EXISTS] [catalog_name.][db_name.]model_name
#Example I: drop model
DROP MODEL `my_model`
DROP MODEL IF EXISTS `my_model`</pre>
</div></div><ul class="ak-ul"><li><p>If IF EXISTS is provided and the model or version doesn’t exist, nothing happens.</p></li></ul><p><br/></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default"># Show Models Syntax
SHOW MODELS [ ( FROM | IN ) [catalog_name.]database_name ]
[ [NOT] LIKE <sql_like_pattern> ]</pre>
</div></div><ul class="ak-ul"><li><p>Show all models in a catalog / database. </p></li><li><p>A pattern can also be provided to filter models.</p></li></ul><p><br/></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default"># Describe Model Syntax
{ DESCRIBE | DESC } MODEL [catalog_name.][database_name.]model_name</pre>
</div></div><ul class="ak-ul"><li><p>Describe a model to show input/output schemas</p></li></ul><p><br/></p><p><span class="prismjs cc-1j55rvq"><code class="language-"><span class=""># Show create model
</span>SHOW CREATE MODEL [catalog_name.][database_name.]model_name</code></span></p><ul class="ak-ul"><li><p>Show all model information including properties, comments etc</p></li></ul><h2 id="FLIP437:SupportMLModelsinFlinkSQL-ModelFunctions">Model Functions</h2><p><span style="color: rgb(23,43,77);">Once a ML model is created in the Flink SQL catalog, we can run streaming (batch) prediction or evaluation against new data with the model. We propose to have the table value function and table aggregation function to support model predictions and evaluations respectively in Flink SQL jobs. Here are the query examples:</span></p><p><br/></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default"># Model Prediction
SELECT f1, f2, label FROM ML_PREDICT(`my_data`, `classifier_model`, DESCRIPTOR(f1, f2))
# Prediction with named arguments
SELECT f1, f2, label FROM
ML_PREDICT(
input => `my_data`,
model => `classifier_model`,
args => DESCRIPTOR(f1, f2)
)
# Model Evaluation
SELECT * FROM ML_EVALUATE(`eval_data`, `classifier_model`, DESCRIPTOR(f1, f2))
# Evaluation with named arguments
SELECT * FROM ML_EVALUATE(
input => `eval_data`,
model => `classifier_model`,
args => DESCRIPTOR(f1, f2)
)</pre>
</div></div><p>We use polymorphic table function which take a whole table and model as input even if support user-defined ones are not yet available in Flink.</p><p>Note that this FLIP defines how Polymorphic Table Functions (PTF) syntax should be defined for Flink SQL going forward. Even before FLIP-440 is implemented, we introduce the concise Oracle-inspired syntax for built-in functions. Meaning we are dropping <code>TABLE(...)</code> by newest additions to Calcite and use (expanded) <code>SqlIdentifier</code> for declaring tables and models.</p><p>Meanwhile, unlike other table or table aggregation functions, the function signature is determined by the underlying model invoked, which means the functions can not be resolved or constructed based on the function name (which is the default way in Flink). One way to address this issue is to introduce a specialized user defined function interface (called ModelFunction), which holds the catalog model information during the job lifecycle.</p><h1 id="FLIP437:SupportMLModelsinFlinkSQL-Built-inMLProviderSupport">Built-in ML Provider Support</h1><p>For remote model providers, we can start with popular vendors such as OpenAI, AzureML, AWS SageMaker.</p><p><br/></p><p><span style="font-size: 24.0px;letter-spacing: -0.01em;">Compatibility, Deprecation, and Migration Plan</span></p><ul><li>The new PTF syntax is only an additional alternative, we keep the old PTF syntax as an "extended syntax" officially supported. It's not deprecated.</li></ul><h1 id="FLIP437:SupportMLModelsinFlinkSQL-TestPlan">Test Plan</h1><ul><li>All existing tests pass</li><li>Add new unit tests and integration tests for any new code changes</li></ul><p><span style="font-size: 24.0px;letter-spacing: -0.01em;">Rejected Alternatives</span></p><p><em>If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.</em></p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-437%3A+Support+ML+Models+in+Flink+SQL">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=296290787&revisedVersion=6&originalVersion=5">View Changes Online</a>
</div>
</div>Timo Walther2024-03-12T14:53:14ZFLIP-402: Extend ZooKeeper Curator configurationsMatthias Pohltag:cwiki.apache.org,2009:page-283118532-122024-03-28T10:00:40Z2023-12-14T12:57:43Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~mapohl
">Matthias Pohl</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<p><br/></p><div class='plugin-tabmeta-details'><p class="auto-cursor-target"><em>Document the state by adding a label to the FLIP page with one of "discussion", "accepted", "released", "rejected".</em></p><div class="table-wrap"><table class="wrapped confluenceTable"><colgroup><col/><col/></colgroup><tbody><tr><th class="confluenceTh">Discussion thread</th><td class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/gqgs2jlq6bmg211gqtgdn8q5hp5v9l1z" rel="nofollow">https://lists.apache.org/thread/gqgs2jlq6bmg211gqtgdn8q5hp5v9l1z</a></td></tr><tr><th colspan="1" class="confluenceTh">Vote thread</th><td colspan="1" class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/0bobx7y14b17j1rcrccodg990wngoboj" rel="nofollow">https://lists.apache.org/thread/0bobx7y14b17j1rcrccodg990wngoboj</a></td></tr><tr><th class="confluenceTh">JIRA</th><td class="confluenceTd"><div class="content-wrapper"><p>
<span class="jira-issue resolved" data-jira-key="FLINK-33376" >
<a href="https://issues.apache.org/jira/browse/FLINK-33376" class="jira-issue-key"><img class="icon"
src="https://issues.apache.org/jira/secure/viewavatar?size=xsmall&avatarId=21140&avatarType=issuetype"/>FLINK-33376</a>
-
<span class="summary">Extend Curator config option for Zookeeper configuration</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-success
jira-macro-single-issue-export-pdf">Resolved</span>
</span>
</p></div></td></tr><tr><th colspan="1" class="confluenceTh">Release</th><td colspan="1" class="confluenceTd">1.20.0</td></tr></tbody></table></div></div><p>Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).</p><h1 id="FLIP402:ExtendZooKeeperCuratorconfigurations-Motivation">Motivation</h1><p>Apache Curator is used in order to perform interactions with ZooKeeper in <a class="external-link" href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/ha/zookeeper_ha/" rel="nofollow">HA mode</a> for Flink. Current set up misses several configurations options, which could be useful in certain Flink deployments.</p><p>We want to ensure that related available options in Apache Curator are configurable for Flink users. Thus Flink users can have all mechanisms to allow Flink interacts with ZooKeeper. Given list of features could be critical for Flink adoption with ZooKeeper in cloud environment. For example, currently is not possible to use ZooKeeper with authorization mechanism along with Flink.</p><h1 id="FLIP402:ExtendZooKeeperCuratorconfigurations-PublicInterfaces">Public Interfaces</h1><p>There are some new configurations should be exposed for <a class="external-link" href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/ha/zookeeper_ha/#configuration" rel="nofollow" style="text-decoration: none;">high-availability.zookeeper</a> configuration.</p><div class="table-wrap"><table class="relative-table wrapped confluenceTable" style="width: 57.1739%;"><colgroup><col style="width: 44.0515%;"/><col style="width: 14.7483%;"/><col style="width: 41.2002%;"/></colgroup><tbody><tr><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><strong>Proposed option</strong></p></td><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><strong>Configuration</strong><strong> type</strong></p></td><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><strong>Motivation</strong></p></td></tr><tr><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#authorization(java.lang.String,byte%5B%5D)" rel="nofollow" style="text-decoration: none;">high-availability.zookeeper.client.authorization</a></p><p><br/></p></td><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><a class="external-link" href="https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java#L163" rel="nofollow">ConfigOptions#mapType()</a></p></td><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p>Ability to fully utilise given set up of ZooKeeper for environment.</p><p>For example: In certain cases ZooKeeper requires additional Authorization information. For example list of valid <a class="external-link" href="https://zookeeper.apache.org/doc/r3.8.0/zookeeperAdmin.html#:~:text=for%20secure%20authentication.-,zookeeper.ensembleAuthName,-%3A%20(Java%20system%20property" rel="nofollow" style="text-decoration: none;">names for ensemble</a> in order to prevent the accidental connecting to a wrong ensemble.</p></td></tr><tr><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#maxCloseWaitMs(int)" rel="nofollow" style="text-decoration: none;">high-availability.zookeeper.client.max-close-wait-ms</a></p><p><br/></p></td><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><a class="external-link" href="https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java#L116" rel="nofollow">ConfigOptions#intType()</a></p></td><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p>Ability that would enable the user to adjust to different network speeds.</p></td></tr><tr><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#simulatedSessionExpirationPercent(int)" rel="nofollow" style="text-decoration: none;">high-availability.zookeeper.client.simulated-session-expiration-percent</a></p></td><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p><a class="external-link" href="https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java#L116" rel="nofollow">ConfigOptions#intType()</a></p></td><td colspan="1" rowspan="1" style="text-align: left;" class="confluenceTd"><p>Additional checking for Session expiration above what is provided by ZooKeeper.</p></td></tr></tbody></table></div><p>The rest of the options provided by Curator framework are considered as non-useful:</p><ul class="ak-ul"><li><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#canBeReadOnly(boolean)" rel="nofollow" style="text-decoration: none;">canBeReadOnly</a> - allowing to read from the stale ZooKeeper could lead to the inconsistent state on the Flink side, e.g. two active JobManager</p></li><li><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#compressionProvider(org.apache.curator.framework.api.CompressionProvider)" rel="nofollow" style="text-decoration: none;">compressionProvider</a> - since Flink doesn’t store a lot of information in Zookeeper there is no need to provide any compression</p></li><li><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#defaultData(byte%5B%5D)" rel="nofollow" style="text-decoration: none;">defaultData</a> - could be useful for debugging purposes of Curator framework, but seems to be non-needed for Flink</p></li><li><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#dontUseContainerParents()" rel="nofollow" style="text-decoration: none;">dontUseContainerParents</a>/ <a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#useContainerParentsIfAvailable()" rel="nofollow" style="text-decoration: none;">useContainerParentsIfAvailable</a> - this sounds like a property that is useful for Flink's leader election cleanup. But I don't see extra value in exposing the property to the user.</p></li><li><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#namespace(java.lang.String)" rel="nofollow" style="text-decoration: none;">namespace</a> - This one is already in use (see <a class="external-link" href="https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/config/#high-availability-zookeeper-path-root" rel="nofollow">high-availability.zookeeper.path.root</a>)</p></li><li><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#runSafeService(java.util.concurrent.Executor)" rel="nofollow" style="text-decoration: none;">runSafeService</a> - That seems to be a feature that's Flink-specific and shouldn't be handled by the user.</p></li><li><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#schemaSet(org.apache.curator.framework.schema.SchemaSet)" rel="nofollow" style="text-decoration: none;">schemaSet</a> - seems that it shouldn’t be exposed to the end user</p></li><li><p><a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#waitForShutdownTimeoutMs(int)" rel="nofollow" style="text-decoration: none;">waitForShutdownTimeoutMs</a> - considered as an internal Flink logic and shouldn’t be exposed.</p></li></ul><h1 id="FLIP402:ExtendZooKeeperCuratorconfigurations-ProposedChanges">Proposed Changes</h1><p>We should incorporate the aforementioned options and translate configuration values into the corresponding Curator builder calls.</p><p>An issue arises due to a type mismatch between the Flink configuration parameter <a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#authorization(java.lang.String,byte%5B%5D)" rel="nofollow">high-availability.zookeeper.client.authorization</a> and the corresponding Curator method call. The Curator method anticipates an array of <a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/AuthInfo.html" rel="nofollow">AuthInfo</a> (see method <a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#authorization(java.util.List)" rel="nofollow">javadoc</a> with signature authorization#List<AuthInfo>) while the Flink configuration for <a class="external-link" href="https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java#L163" rel="nofollow">ConfigOptions#mapType()</a> provides a different java type - <em>Map<String, String></em>. To resolve this, we suggest the following conversion: Each entry of type <em>Map.Entry<String, String></em> will be transformed into an <a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/AuthInfo.html" rel="nofollow">AuthInfo</a> object with the constructor <code>AuthInfo(String, byte[])</code>. The field <code>entry.key()</code> will serve as the <code>String scheme</code> value, while the field <code>entry.getValue()</code> will be initially converted to a <code>byte[]</code> using the <code>String#getBytes()</code> method. Subsequently, this byte array will be utilized as <code>byte[] auth</code> parameter during the creation of the <a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/AuthInfo.html" rel="nofollow">AuthInfo</a>.</p><h1 id="FLIP402:ExtendZooKeeperCuratorconfigurations-Compatibility,Deprecation,andMigrationPlan">Compatibility, Deprecation, and Migration Plan</h1><p><span style="color: rgb(23,43,77);">N/A</span></p><h1 id="FLIP402:ExtendZooKeeperCuratorconfigurations-TestPlan">Test Plan</h1><p>Simple manual tests will do that given options are well applied.</p><p>For <a class="external-link" href="https://curator.apache.org/apidocs/org/apache/curator/framework/CuratorFrameworkFactory.Builder.html#authorization(java.lang.String,byte%5B%5D)" rel="nofollow" style="text-decoration: none;">high-availability.zookeeper.client.authorization</a> we can add a unit test which validates the conversion between the <em>Map<String, String></em> and <em>AuthInfo[]</em>.</p><h1 id="FLIP402:ExtendZooKeeperCuratorconfigurations-RejectedAlternatives">Rejected Alternatives</h1><h2 id="FLIP402:ExtendZooKeeperCuratorconfigurations-GenericconfigurationforallApacheCuratoroptionsvianamespaces">Generic configuration for all Apache Curator options via namespaces</h2><p>We could think about utilising the namespaces. The FLIP could propose adding namespace support for Apache Curator . E.g. metric high-availability.zookeeper.client.<em><config_option></em> could be translated into the appropriate <em><config_option></em> of the Curator configuration. That would allow to load any parameter supported by these systems.</p><p>Unfortunately Curator connection is configured via Builder pattern, when single configuration is translated into the proper call of the Builder object.</p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-402%3A+Extend+ZooKeeper+Curator+configurations">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=283118532&revisedVersion=12&originalVersion=11">View Changes Online</a>
</div>
</div>Matthias Pohl2023-12-14T12:57:43Z1.20 ReleaseYubin Litag:cwiki.apache.org,2009:page-296291013-112024-03-28T08:17:13Z2024-03-18T02:19:16Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~lixin58688
">Yubin Li</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<h1 id="id-1.20Release-Tableofcontents"><span>Table of contents</span></h1><p><span><style type='text/css'>/*<![CDATA[*/
div.rbtoc1711724892989 {padding: 0px;}
div.rbtoc1711724892989 ul {margin-left: 0px;}
div.rbtoc1711724892989 li {margin-left: 0px;padding-left: 0px;}
/*]]>*/</style><div class='toc-macro rbtoc1711724892989'>
<ul class='toc-indentation'>
<li><a href='#id-1.20Release-Tableofcontents'>Table of contents</a></li>
<li><a href='#id-1.20Release-JiraBoards'>Jira Boards</a></li>
<li><a href='#id-1.20Release-Syncmeeting'>Sync meeting</a></li>
<li><a href='#id-1.20Release-Timeline'>Timeline</a></li>
<li><a href='#id-1.20Release-Highlightfeatures'>Highlight features</a></li>
<li><a href='#id-1.20Release-Features'>Features</a>
<ul class='toc-indentation'>
<li><a href='#id-1.20Release-Legend'>Legend</a>
<ul class='toc-indentation'>
<li><a href='#id-1.20Release-State'>State</a></li>
<li><a href='#id-1.20Release-X-Teamverification'>X-Team verification</a></li>
<li><a href='#id-1.20Release-FeatureStage'>Feature Stage</a></li>
</ul>
</li>
<li><a href='#id-1.20Release-Summary'>Summary</a></li>
<li><a href='#id-1.20Release-List'>List</a></li>
</ul>
</li>
<li><a href='#id-1.20Release-Status/Follow-ups'>Status / Follow-ups</a>
<ul class='toc-indentation'>
<li><a href='#id-1.20Release-03/27/24'>03/27/24</a></li>
</ul>
</li>
</ul>
</div></span></p><h1 id="id-1.20Release-JiraBoards"><span>Jira Boards</span></h1><p><span>Flink 1.20 Burndown: <a class="external-link" href="https://issues.apache.org/jira/secure/RapidBoard.jspa?projectKey=FLINK&rapidView=604" rel="nofollow">1.20 Burndown - Agile Board - ASF JIRA (apache.org)</a></span></p><h1 id="id-1.20Release-Syncmeeting">Sync meeting</h1><ul style="list-style-type: square;"><li>The release sync will start from April 9, 2024, at 10am (UTC+2) and 4pm (UTC+8). The release sync happens bi-weekly at first, and will be adjusted to weekly as we approaching the feature freeze date. </li><li>Feel free to join on <a class="external-link" href="https://meet.google.com/mtj-huez-apu" rel="nofollow">Google Meet</a>, <span style="color: rgb(23,43,77);">and attendees are encouraged to fill out the topics to be discussed at the bottom of this page a day in advance to make it easier for everyone to understand the background of the topic. </span></li></ul><h1 id="id-1.20Release-Timeline">Timeline</h1><ul><li><strong>Feature Freeze</strong><br/><ul><li><span style="color: rgb(23,43,77);text-decoration: none;">June 15, 2024, <span style="color: rgb(34,34,51);">00:00 CEST(UTC+2)</span> </span></li></ul></li><li><strong>Release</strong><ul><li>End of July 2024</li></ul></li></ul><p><br/></p><h1 id="id-1.20Release-Highlightfeatures">Highlight features</h1><p><em>please feel free to add/suggest.</em></p><h1 id="id-1.20Release-Features"><span>Features</span></h1><p><span>List of features announced by contributors and committers that are likely to be ready for the feature freeze:n</span></p><p><span><strong>NOTICE:</strong> </span><em>It's preferred if only <strong>new features</strong> end up there and <strong>not all bugs/tasks separately</strong>, so that the page is not over bloated. Of course, unless fixing a bug is a really big or important one equivalent to implementing a completely new feature. A good rule of thumb would be that each entry in the page could (but does not have to) be later on included in a release blog post.</em></p><h2 id="id-1.20Release-Legend">Legend</h2><h3 id="id-1.20Release-State">State</h3><div class="table-wrap"><table class="wrapped confluenceTable"><colgroup><col/><col/><col/></colgroup><tbody><tr><th class="confluenceTh">symbol</th><th class="confluenceTh">meaning</th><th class="confluenceTh">comment</th></tr><tr><td colspan="1" class="confluenceTd"><img class="emoticon emoticon-laugh" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/biggrin.svg" data-emoticon-name="laugh" alt="(big grin)" /></td><td colspan="1" class="confluenceTd">validated</td><td colspan="1" class="confluenceTd">through cross team testing</td></tr><tr><td class="confluenceTd"><img class="emoticon emoticon-tick" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/check.svg" data-emoticon-name="tick" alt="(tick)" /></td><td class="confluenceTd">done</td><td class="confluenceTd">well documented with a complete test coverage</td></tr><tr><td class="confluenceTd"><img class="emoticon emoticon-green-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_green.svg" data-emoticon-name="green-star" alt="(green star)" /></td><td class="confluenceTd"><span class="inline-comment-marker" data-ref="894d717c-441f-4731-b5cf-b179eacec65a">will make it</span></td><td class="confluenceTd">there is no reason this effort should not go into 1.19</td></tr><tr><td class="confluenceTd"><img class="emoticon emoticon-yellow-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_yellow.svg" data-emoticon-name="yellow-star" alt="(star)" /></td><td class="confluenceTd">in danger</td><td class="confluenceTd">there are some concerns the effort could be ready for the feature freeze of 1.19</td></tr><tr><td class="confluenceTd"><img class="emoticon emoticon-red-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_red.svg" data-emoticon-name="red-star" alt="(red star)" /></td><td class="confluenceTd">very unlikely</td><td class="confluenceTd">there are severe concerns the effort could make it to 1.19</td></tr><tr><td class="confluenceTd"><img class="emoticon emoticon-minus" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/forbidden.svg" data-emoticon-name="minus" alt="(minus)" /></td><td class="confluenceTd">won't make it</td><td class="confluenceTd">it was decided against adding this for the 1.19 release. working on the effort has been stopped.</td></tr><tr><td class="confluenceTd"><img class="emoticon emoticon-question" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/help_16.svg" data-emoticon-name="question" alt="(question)" /></td><td class="confluenceTd">state unclear</td><td class="confluenceTd"><br/></td></tr><tr><td colspan="1" class="confluenceTd"><img class="emoticon emoticon-blue-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_blue.svg" data-emoticon-name="blue-star" alt="(blue star)" /></td><td colspan="1" class="confluenceTd">independent</td><td colspan="1" class="confluenceTd">as the artifact could be released independent of Apache Flink</td></tr></tbody></table></div><h3 id="id-1.20Release-X-Teamverification">X-Team verification</h3><div class="table-wrap"><table class="wrapped confluenceTable"><colgroup><col/><col/></colgroup><tbody><tr><th class="confluenceTh">symbol</th><th class="confluenceTh">meaning</th></tr><tr><td class="confluenceTd"><img class="emoticon emoticon-tick" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/check.svg" data-emoticon-name="tick" alt="(tick)" /></td><td class="confluenceTd">done</td></tr><tr><td colspan="1" class="confluenceTd"><img class="emoticon emoticon-blue-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_blue.svg" data-emoticon-name="blue-star" alt="(blue star)" /></td><td colspan="1" class="confluenceTd">not required</td></tr></tbody></table></div><h3 id="id-1.20Release-FeatureStage">Feature Stage</h3><p>Please align with the list on the Apache Flink Roadmap (<a class="external-link" href="https://flink.apache.org/roadmap.html" rel="nofollow">https://flink.apache.org/roadmap.html</a>).</p><ul style="text-decoration: none;"><li><strong>MVP:</strong> Have a look, consider whether this can help you in the future.</li><li><strong>Beta:</strong> You can benefit from this, but you should carefully evaluate the feature.</li><li><strong>Ready and Evolving:</strong> Ready to use in production, but be aware you may need to make some adjustments to your application and setup in the future, when you upgrade Flink.</li><li><strong>Stable:</strong> Unrestricted use in production</li><li><strong>Reaching End-of-Life:</strong> Stable, still feel free to use, but think about alternatives. Not a good match for new long-lived projects.</li><li><strong>Deprecated:</strong> Start looking for alternatives now<span style="letter-spacing: 0.0px;"> </span></li></ul><h2 id="id-1.20Release-Summary"><span class="inline-comment-marker" data-ref="f90724ca-d05f-4d70-9b11-6d9637b2cd11">Summary</span></h2><p><em>Numbers are based on the items in the list below, not on the tickets </em></p><div class="table-wrap"><table class="wrapped relative-table confluenceTable"><colgroup><col style="width: 179.0px;"/><col style="width: 87.0px;"/><col style="width: 87.0px;"/><col style="width: 92.0px;"/><col style="width: 81.0px;"/><col style="width: 81.0px;"/><col style="width: 82.0px;"/><col style="width: 81.0px;"/><col style="width: 77.0px;"/><col style="width: 84.0px;"/><col style="width: 365.0px;"/></colgroup><tbody><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><em><img class="emoticon emoticon-laugh" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/biggrin.svg" data-emoticon-name="laugh" alt="(big grin)" /></em></td><td class="confluenceTd"><em><img class="emoticon emoticon-tick" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/check.svg" data-emoticon-name="tick" alt="(tick)" /> </em></td><td class="confluenceTd"><img class="emoticon emoticon-green-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_green.svg" data-emoticon-name="green-star" alt="(green star)" /></td><td class="confluenceTd"><img class="emoticon emoticon-yellow-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_yellow.svg" data-emoticon-name="yellow-star" alt="(star)" /></td><td class="confluenceTd"><img class="emoticon emoticon-red-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_red.svg" data-emoticon-name="red-star" alt="(red star)" /></td><td class="confluenceTd"><img class="emoticon emoticon-minus" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/forbidden.svg" data-emoticon-name="minus" alt="(minus)" /></td><td class="confluenceTd"><img class="emoticon emoticon-question" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/help_16.svg" data-emoticon-name="question" alt="(question)" /></td><td class="confluenceTd"><img class="emoticon emoticon-blue-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_blue.svg" data-emoticon-name="blue-star" alt="(blue star)" /></td><td class="confluenceTd">∑</td><td class="confluenceTd">Remaining weeks</td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><em><br/></em></td><td class="confluenceTd"><em><br/></em></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr></tbody></table></div><h2 class="auto-cursor-target" id="id-1.20Release-List">List</h2><p>Feel free to add categories.</p><div class="table-wrap"><table class="relative-table wrapped confluenceTable" style="width: 99.1822%;"><colgroup><col style="width: 11.125%;"/><col style="width: 11.837%;"/><col style="width: 10.235%;"/><col style="width: 4.71698%;"/><col style="width: 7.29797%;"/><col style="width: 3.64899%;"/><col style="width: 4.22748%;"/><col style="width: 5.20648%;"/><col style="width: 7.03097%;"/><col style="width: 24.2969%;"/><col style="width: 10.3685%;"/></colgroup><tbody><tr><th colspan="11" class="confluenceTh">Runtime</th></tr><tr><th class="confluenceTh"><span class="inline-comment-marker" data-ref="b73c761f-b3fc-4905-8e48-f5a5266a9a79">Name/JIRA Issue</span></th><th class="confluenceTh">Responsible Contributor</th><th class="confluenceTh">Reviewer / committer available</th><th class="confluenceTh">Feature Stage</th><th class="confluenceTh">Note</th><th class="confluenceTh">State</th><th class="confluenceTh"><strong>%</strong></th><th class="confluenceTh">Updated</th><th class="confluenceTh">Implemented</th><th class="confluenceTh">Documented</th><th class="confluenceTh">X-team verified</th></tr><tr><th class="confluenceTh"><span class="inline-comment-marker" data-ref="b73c761f-b3fc-4905-8e48-f5a5266a9a79">Shuffle</span></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><strong><br/></strong></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><th class="confluenceTh">Coordination</th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><strong><br/></strong></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><strong>State & Checkpoint</strong></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><strong title=""><br/></strong></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td></tr><tr><td class="confluenceTd"><div class="content-wrapper"><p><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-306%3A+Unified+File+Merging+Mechanism+for+Checkpoints">FLIP-306: Unified File Merging Mechanism for Checkpoints</a></p><p>
<span class="jira-issue" data-jira-key="FLINK-32070" data-client-id="SINGLE_5aa69414-a9e9-3523-82ec-879b028fb15b_296291013_anonymous" >
<a href="https://issues.apache.org/jira/browse/FLINK-32070" class="jira-issue-key"><span
class="aui-icon aui-icon-wait issue-placeholder"></span>FLINK-32070</a>
-
<span class="summary">Getting issue details...</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-default issue-placeholder">STATUS</span>
</span>
</p></div></td><td class="confluenceTd"><div class="content-wrapper"><p><a class="confluence-userlink user-mention" data-username="zakelly" href="https://cwiki.apache.org/confluence/display/~zakelly" data-linked-resource-id="181311105" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Zakelly Lan</a> </p><p><a class="confluence-userlink user-mention" data-username="fredialei" href="https://cwiki.apache.org/confluence/display/~fredialei" data-linked-resource-id="211886241" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Yanfei Lei</a> </p><p><a class="confluence-userlink user-mention" data-username="masteryhx" href="https://cwiki.apache.org/confluence/display/~masteryhx" data-linked-resource-id="181309527" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Hangxiang Yu</a> </p></div></td><td class="confluenceTd"><div class="content-wrapper"><p><a class="confluence-userlink user-mention" data-username="fredialei" href="https://cwiki.apache.org/confluence/display/~fredialei" data-linked-resource-id="211886241" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Yanfei Lei</a></p><p><a class="confluence-userlink user-mention" data-username="masteryhx" href="https://cwiki.apache.org/confluence/display/~masteryhx" data-linked-resource-id="181309527" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Hangxiang Yu</a> </p></div></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><img class="emoticon emoticon-green-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_green.svg" data-emoticon-name="green-star" alt="(green star)" /></td><td class="confluenceTd"><br/></td><td class="confluenceTd">2024-03-19</td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><div class="content-wrapper" title=""><p><strong>REST</strong></p></div></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><div class="content-wrapper" title=""><p><br/></p></div></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><strong title=""><br/></strong></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7" title="Background color : Light grey 100%"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><div class="content-wrapper" title=""><p><strong>Web UI</strong></p></div></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><div class="content-wrapper" title=""><p><br/></p></div></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><strong title=""><br/></strong></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><br/></td><td class="highlight-#f4f5f7 confluenceTd" data-highlight-colour="#f4f5f7"><br/></td></tr><tr><td class="confluenceTd"><div class="content-wrapper"><p><a href="https://cwiki.apache.org/confluence/x/agrPEQ" rel="nofollow">FLIP-441: Show the JobType and remove Execution Mode on Flink WebUI</a></p><p>
<span class="jira-issue" data-jira-key="FLINK-29481" data-client-id="SINGLE_5aa69414-a9e9-3523-82ec-879b028fb15b_296291013_anonymous" >
<a href="https://issues.apache.org/jira/browse/FLINK-29481" class="jira-issue-key"><span
class="aui-icon aui-icon-wait issue-placeholder"></span>FLINK-29481</a>
-
<span class="summary">Getting issue details...</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-default issue-placeholder">STATUS</span>
</span>
</p></div></td><td class="confluenceTd"><div class="content-wrapper"><p><a class="confluence-userlink user-mention" data-username="fanrui" href="https://cwiki.apache.org/confluence/display/~fanrui" data-linked-resource-id="123900645" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Rui Fan</a> </p></div></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><img class="emoticon emoticon-green-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_green.svg" data-emoticon-name="green-star" alt="(green star)" /></td><td class="confluenceTd"><br/></td><td class="confluenceTd">2024-03-27</td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><th colspan="11" class="confluenceTh">SQL<strong><br/></strong></th></tr><tr><th class="confluenceTh">Name/JIRA Issue</th><th class="confluenceTh">Responsible Contributor</th><th class="confluenceTh">Reviewer / committer available</th><th class="confluenceTh">Feature Stage</th><th class="confluenceTh">Note</th><th class="confluenceTh">State</th><th class="confluenceTh"><strong>%</strong></th><th class="confluenceTh">Updated</th><th class="confluenceTh">Implemented</th><th class="confluenceTh">Documented</th><th class="confluenceTh">X-team verified</th></tr><tr><td class="confluenceTd"><div class="content-wrapper"><p><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-436%3A+Introduce+Catalog-related+Syntax">FLIP-436: Introduce Catalog-related Syntax</a></p><p>
<span class="jira-issue" data-jira-key="FLINK-34914" data-client-id="SINGLE_5aa69414-a9e9-3523-82ec-879b028fb15b_296291013_anonymous" >
<a href="https://issues.apache.org/jira/browse/FLINK-34914" class="jira-issue-key"><span
class="aui-icon aui-icon-wait issue-placeholder"></span>FLINK-34914</a>
-
<span class="summary">Getting issue details...</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-default issue-placeholder">STATUS</span>
</span>
</p></div></td><td class="confluenceTd"><div class="content-wrapper"><p><a class="confluence-userlink user-mention" data-username="lixin58688" href="https://cwiki.apache.org/confluence/display/~lixin58688" data-linked-resource-id="191336734" data-linked-resource-version="2" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Yubin Li</a> </p></div></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><img class="emoticon emoticon-green-star" src="https://cwiki.apache.org/confluence/s/-577jzc/8804/10mvnxf/_/images/icons/emoticons/star_green.svg" data-emoticon-name="green-star" alt="(green star)" /></td><td class="confluenceTd"><br/></td><td class="confluenceTd">2024-03-28</td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><th class="confluenceTh">Connectors</th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><strong><br/></strong></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><th class="confluenceTh">Resource Management</th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><strong><br/></strong></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th><th class="confluenceTh"><br/></th></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><th colspan="11" class="confluenceTh"><div class="content-wrapper"><p>Runtime</p></div></th></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><div class="content-wrapper"><p><strong>Misc</strong></p></div></td><td class="confluenceTd"><div class="content-wrapper"><p><br/></p></div></td><td class="confluenceTd"><div class="content-wrapper"><p><br/></p></div></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><strong><br/></strong></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td><td class="confluenceTd"><br/></td></tr></tbody></table></div><h1 class="auto-cursor-target" id="id-1.20Release-Status/Follow-ups">Status / Follow-ups</h1><h2 id="id-1.20Release-03/27/24">03/27/24</h2><ul><li><span>Daily work</span><ul><li>CI (Failures reported in #build channel and not reported)</li><li>Benchmark (Check if there's regression in the #flink-dev-benchmarks channel)</li></ul></li></ul>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/1.20+Release">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=296291013&revisedVersion=11&originalVersion=10">View Changes Online</a>
</div>
</div>Yubin Li2024-03-18T02:19:16ZFLIP-441: Show the JobType and remove Execution Mode on Flink WebUIRui Fantag:cwiki.apache.org,2009:page-298781290-42024-03-28T07:27:00Z2024-03-26T07:31:24Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~fanrui
">Rui Fan</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<p><em><br/></em></p><div class='plugin-tabmeta-details'><div class="table-wrap"><table class="wrapped relative-table confluenceTable" style="width: 40.9672%;"><colgroup><col style="width: 23.2836%;"/><col style="width: 76.6893%;"/></colgroup><tbody><tr><th class="confluenceTh">Discussion thread</th><td class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/0s52w17w24x7m2zo6ogl18t1fy412vcd" rel="nofollow">https://lists.apache.org/thread/0s52w17w24x7m2zo6ogl18t1fy412vcd</a></td></tr><tr><th colspan="1" class="confluenceTh">Vote thread</th><td colspan="1" class="confluenceTd"><br/></td></tr><tr><th class="confluenceTh">JIRA</th><td class="confluenceTd"><div class="content-wrapper"><p>
<span class="jira-issue" data-jira-key="FLINK-29481" >
<a href="https://issues.apache.org/jira/browse/FLINK-29481" class="jira-issue-key"><img class="icon"
src="https://issues.apache.org/jira/secure/viewavatar?size=xsmall&avatarId=21140&avatarType=issuetype"/>FLINK-29481</a>
-
<span class="summary">Show JobType on WebUI</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-default
jira-macro-single-issue-export-pdf">Open</span>
</span>
</p></div></td></tr><tr><th colspan="1" class="confluenceTh">Release</th><td colspan="1" class="confluenceTd"><br/></td></tr></tbody></table></div></div><p><br/></p><p><span><style type='text/css'>/*<![CDATA[*/
div.rbtoc1711724893146 {padding: 0px;}
div.rbtoc1711724893146 ul {margin-left: 0px;}
div.rbtoc1711724893146 li {margin-left: 0px;padding-left: 0px;}
/*]]>*/</style><div class='toc-macro rbtoc1711724893146'>
<ul class='toc-indentation'>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-1.Motivation'>1. Motivation</a>
<ul class='toc-indentation'>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-ClarificationonExecutionmode.'>Clarification on Execution mode.</a></li>
</ul>
</li>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.PublicInterfaces'>2. Public Interfaces</a>
<ul class='toc-indentation'>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.1AddjobTypeinrestapi'>2.1 Add jobType in rest api</a></li>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.2ShowJobTypeonFlinkWebUI'>2.2 Show JobType on Flink Web UI</a></li>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.3RemoveExecutionmodeinFlinkWebUI'>2.3 Remove Execution mode in Flink WebUI</a></li>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.4Deprecateexecution-modeinExecutionConfigInforelatedrestapi.'>2.4 Deprecate execution-mode in ExecutionConfigInfo related rest api.</a></li>
</ul>
</li>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-3.ProposedChanges'>3. Proposed Changes</a></li>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-4.Compatibility,Deprecation,andMigrationPlan'>4. Compatibility, Deprecation, and Migration Plan</a></li>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-5.TestPlan'>5. Test Plan</a></li>
<li><a href='#FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-6.RejectedAlternatives'>6. Rejected Alternatives</a></li>
</ul>
</div></span></p><h1 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-1.Motivation">1. Motivation</h1><p>Currently, the jobType has 2 types in Flink: STREAMING and BATCH. They work on completely different principles, such as: scheduler, shuffle, join, etc. These differences lead to different troubleshooting processes, so when users are maintaining a job or troubleshooting, it's needed to know whether the current job is a STREAMING or BATCH job. Unfortunately, Flink WebUI doesn't expose it to the users so far.</p><p>Also, ExecutionMode is related to DataSet api, it has been marked as @Deprecated in
<span class="jira-issue resolved" data-jira-key="FLINK-32558" >
<a href="https://issues.apache.org/jira/browse/FLINK-32558" class="jira-issue-key"><img class="icon"
src="https://issues.apache.org/jira/secure/viewavatar?size=xsmall&avatarId=21146&avatarType=issuetype"/>FLINK-32558</a>
-
<span class="summary">Properly deprecate DataSet API</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-success
jira-macro-single-issue-export-pdf">Closed</span>
</span>
(1.18), but it's still shown in Flink WebUI.</p><h2 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-ClarificationonExecutionmode.">Clarification on Execution mode.</h2><ul><li><a class="external-link" href="https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/dev/datastream/execution_mode/" rel="nofollow">Flink 1.19 website</a> still mentions the Execution mode, but it actually matches the <a class="external-link" href="https://github.com/apache/flink/blob/f31c128bfc457b64dd7734f71123b74faa2958ba/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobType.java#L22" rel="nofollow">JobType</a> in the Flink code. Both of them have 2 types: STREAMING and BATCH.</li><li>execution.runtime-mode can be set to 3 types: STREAMING, BATCH and AUTOMATIC.<ul><li>But the jobType will be inferred as STREAMING or BATCH when execution.runtime-mode is set to AUTOMATIC.</li></ul></li><li>The ExecutionMode I describe is: <a class="external-link" href="https://github.com/apache/flink/blob/f31c128bfc457b64dd7734f71123b74faa2958ba/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java#L54" rel="nofollow">code link</a> , as we can see, ExecutionMode has 4 enums: PIPELINED, PIPELINED_FORCED, BATCH and BATCH_FORCED. And we can see a flink streaming job from Flink WebUI, the Execution mode is PIPELINE instead of STREAMING.</li><li>What this proposal wants to do is to remove the ExecutionMode with four enumerations on Flink WebUI and introduce the JobType with two enumerations (STREAMING or BATCH).<ul><li>STREAMING or BATCH is clearer and more accurate for users.</li></ul></li></ul><p><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image" draggable="false" width="1269" src="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-28_15-12-26.png?version=1&modificationDate=1711609947000&api=v2" data-image-src="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-28_15-12-26.png?version=1&modificationDate=1711609947000&api=v2" data-unresolved-comment-count="0" data-linked-resource-id="298781400" data-linked-resource-version="1" data-linked-resource-type="attachment" data-linked-resource-default-alias="image-2024-3-28_15-12-26.png" data-base-url="https://cwiki.apache.org/confluence" data-linked-resource-content-type="image/png" data-linked-resource-container-id="298781290" data-linked-resource-container-version="4" alt=""></span></p><p><br/></p><p>I propose 4 public changes in this FLIP:</p><ul><li>Add jobType in rest api</li><li>Show the JobType on Flink WebUI properly</li><li>Remove Execution mode in Flink WebUI</li><li>Deprecate execution-mode in ExecutionConfigInfo related rest api(Remove it in flink 2.x)</li></ul><p>POC branch: <a class="external-link" href="https://github.com/1996fanrui/flink/tree/flip-441/poc" rel="nofollow">https://github.com/1996fanrui/flink/tree/flip-441/poc</a> (Latest 2 commits)</p><h1 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.PublicInterfaces">2. Public Interfaces</h1><h2 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.1AddjobTypeinrestapi">2.1 Add jobType in rest api</h2><p>As Zhuzhu suggested in this <a class="external-link" href="https://github.com/apache/flink/pull/21840#discussion_r1138368567" rel="nofollow">comment</a>: jobType is an automatically derived property instead of a user setting. Therefore, I would not put it in ExecutionConfig.</p><p>We can add jobType in JobDetailsInfo instead of ExecutionConfig.</p><p>The corresponding rest api is /jobs/:jobid with GET request, the json key is "job-type".</p><h2 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.2ShowJobTypeonFlinkWebUI">2.2 Show JobType on Flink Web UI</h2><p><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image" draggable="false" width="1576" src="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-27_13-42-48.png?version=1&modificationDate=1711518170000&api=v2" data-image-src="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-27_13-42-48.png?version=1&modificationDate=1711518170000&api=v2" data-unresolved-comment-count="0" data-linked-resource-id="298781320" data-linked-resource-version="1" data-linked-resource-type="attachment" data-linked-resource-default-alias="image-2024-3-27_13-42-48.png" data-base-url="https://cwiki.apache.org/confluence" data-linked-resource-content-type="image/png" data-linked-resource-container-id="298781290" data-linked-resource-container-version="4" alt=""></span></p><p><br/></p><h2 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.3RemoveExecutionmodeinFlinkWebUI">2.3 Remove Execution mode in Flink WebUI</h2><p><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image" draggable="false" height="250" src="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-27_13-46-54.png?version=1&modificationDate=1711518416000&api=v2" data-image-src="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-27_13-46-54.png?version=1&modificationDate=1711518416000&api=v2" data-unresolved-comment-count="0" data-linked-resource-id="298781321" data-linked-resource-version="1" data-linked-resource-type="attachment" data-linked-resource-default-alias="image-2024-3-27_13-46-54.png" data-base-url="https://cwiki.apache.org/confluence" data-linked-resource-content-type="image/png" data-linked-resource-container-id="298781290" data-linked-resource-container-version="4" alt=""></span></p><h2 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-2.4Deprecateexecution-modeinExecutionConfigInforelatedrestapi.">2.4 Deprecate execution-mode in ExecutionConfigInfo related rest api.</h2><p>Mark execution-mode as @Deprecate in ExecutionConfigInfo. ( flink 1.20)</p><p>Remove it in flink 2.0 or 2.1</p><h1 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-3.ProposedChanges">3. Proposed Changes</h1><ul><li>Adding getJobType for AccessExecutionGraph interface:<ul><li>DefaultExecutionGraph and ArchivedExecutionGraph need to add jobType field.</li></ul></li></ul><h1 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-4.Compatibility,Deprecation,andMigrationPlan">4. Compatibility, Deprecation, and Migration Plan</h1><p>All WebUI changes take effect for History server, the history server might show the job overview with old flink version. These jobs don't have jobType in the archived files of History server, we can hide the JobType on WebUI when json doesn't have job-type.</p><p><em>*ngIf="jobDetail['job-type'] != null" is easy to hide it.</em></p><p><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image" draggable="false" height="69" src="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-27_14-12-2.png?version=1&modificationDate=1711519923000&api=v2" data-image-src="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-27_14-12-2.png?version=1&modificationDate=1711519923000&api=v2" data-unresolved-comment-count="0" data-linked-resource-id="298781323" data-linked-resource-version="1" data-linked-resource-type="attachment" data-linked-resource-default-alias="image-2024-3-27_14-12-2.png" data-base-url="https://cwiki.apache.org/confluence" data-linked-resource-content-type="image/png" data-linked-resource-container-id="298781290" data-linked-resource-container-version="4" alt=""></span></p><h1 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-5.TestPlan">5. Test Plan</h1><ul><li>Unit test</li><li>Test running job manually, check if the JobType is expected when <span style="color: rgb(0,0,0);">execution.runtime-mode</span> is STREAMING, BATCH and AUTOMATIC</li><li>Test history server manually<ul><li>Check if JobType is expected when <span style="color: rgb(0,0,0);">execution.runtime-mode</span> is STREAMING, BATCH and AUTOMATIC</li><li>Check if JobType is hidden when job version before flink-1.20</li></ul></li></ul><h1 id="FLIP441:ShowtheJobTypeandremoveExecutionModeonFlinkWebUI-6.RejectedAlternatives">6. Rejected Alternatives</h1><ul style="list-style-type: square;"><li>Show the jobType in Execution Configuration<br/><ul style="list-style-type: square;"><li>jobType is an automatically derived property instead of a user setting.</li></ul></li></ul><p><br/></p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-441%3A+Show+the+JobType+and+remove+Execution+Mode+on+Flink+WebUI">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=298781290&revisedVersion=4&originalVersion=3">View Changes Online</a>
</div>
</div>Rui Fan2024-03-26T07:31:24ZFLIP-441: Show the JobType and remove Execution Mode on Flink WebUI > image-2024-3-28_15-12-26.pngRui Fantag:cwiki.apache.org,2009:attachment-298781400-12024-03-28T07:12:27Z2024-03-28T07:12:27Z<div class="feed"> <p>File
<b>attached</b> by
<a href=" https://cwiki.apache.org/confluence/display/~fanrui
">Rui Fan</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<p>
<span class="aui-icon content-type-attachment-image" title="PNG File">PNG File</span> <a href="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-28_15-12-26.png?version=1&modificationDate=1711609947000&api=v2">image-2024-3-28_15-12-26.png</a> <span class="smalltext">(138 kB)</span>
<br/>
</p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/pages/viewpageattachments.action?pageId=298781290">View Attachments</a>
</div>
</div>Rui Fan2024-03-28T07:12:27ZFLIP-428: Fault Tolerance/Rescale Integration for Disaggregated StateJinzhong Litag:cwiki.apache.org,2009:page-293046865-122024-03-28T02:43:34Z2024-02-23T02:59:03Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~lijinzhong
">Jinzhong Li</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<div class="table-wrap"><table class="wrapped confluenceTable" style="letter-spacing: 0.0px;"><thead class=""><tr class=""><th class="confluenceTh">Discussion thread</th><td class="confluenceTd"><p><a class="external-link" href="https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0" rel="nofollow">https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0</a></p><p><a class="external-link" href="https://lists.apache.org/thread/vr8f91p715ct4lop6b3nr0fh4z5p312b" rel="nofollow">https://lists.apache.org/thread/vr8f91p715ct4lop6b3nr0fh4z5p312b</a></p></td></tr></thead><colgroup class=""><col class=""/><col class=""/></colgroup><tbody class=""><tr class=""><th colspan="1" class="confluenceTh">Vote thread</th><td colspan="1" class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/j7015yymmdy9zp9vr3gg7qpmzjw0o7y2" rel="nofollow">https://lists.apache.org/thread/j7015yymmdy9zp9vr3gg7qpmzjw0o7y2</a></td></tr><tr class=""><th class="confluenceTh">JIRA</th><td class="confluenceTd"><br/></td></tr><tr class=""><th colspan="1" class="confluenceTh">Release</th><td colspan="1" class="confluenceTd">2.0</td></tr></tbody></table></div><p>Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).</p><p><style type='text/css'>/*<![CDATA[*/
div.rbtoc1711724893174 {padding: 0px;}
div.rbtoc1711724893174 ul {margin-left: 0px;}
div.rbtoc1711724893174 li {margin-left: 0px;padding-left: 0px;}
/*]]>*/</style><div class='toc-macro rbtoc1711724893174'>
<ul class='toc-indentation'>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-BackgroundandMotivation'>Background and Motivation</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-High-levelOverview'>High-level Overview</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-ProposedChanges'>Proposed Changes</a>
<ul class='toc-indentation'>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Checkpoint'>Checkpoint</a>
<ul class='toc-indentation'>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Keystepsincheckpointsync&asyncphase'>Key steps in checkpoint sync & async phase</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Filemanagement'>File management</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Checkpointcomplete/subsume/abort'>Checkpoint complete/subsume/abort</a></li>
</ul>
</li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Restore'>Restore</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Rescaling'>Rescaling</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Savepoint'>Savepoint</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-RelatedFeaturesSupport'>Related Features Support</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Mid/longtermfollowupwork'>Mid/long term follow up work</a></li>
</ul>
</li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Compatibility,Deprecation,andMigrationPlan'>Compatibility, Deprecation, and Migration Plan</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-TestPlan'>Test Plan</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-RejectedAlternatives'>Rejected Alternatives</a></li>
<li><a href='#FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Reference'>Reference</a></li>
</ul>
</div></p><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p><span style="color: rgb(51,51,51);">This is a<span> </span></span><span style="color: rgb(23,43,77);">sub-FLIP for the disaggregated state management and its related work, please read the<span> </span><a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">FLIP-423</a><span> </span>first to know the whole story.</span></p></div></div><h1 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-BackgroundandMotivation">Background and Motivation</h1><p class="ne-p"><span class="ne-text">As outlined in <a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">FLIP-423</a> [1] and <a href="https://cwiki.apache.org/confluence/x/T4p3EQ" rel="nofollow">FLIP-427</a> [2], we</span><span class="ne-text"> proposed </span><span class="ne-text">to disaggregate StateManagement and introduced a disaggregated state storage named ForSt, which evolves from RocksDB. Within the new framework, where the primary storage is placed on the remote file system,</span><span class="ne-text"> several challenges emerge when attempting to reuse the existing fault-tolerance mechanisms of local RocksDB:</span></p><ul class="ne-ul"><li><span class="ne-text">Because most remote file system don't support hard-link, ForSt can't </span><span class="ne-text">utilize</span><span class="ne-text"> hard-link to capture a consistent snapshot during checkpoint synchronous phase as rocksdb currently does.</span></li><li><span class="ne-text">The existing file transfer mechanism within RocksDB is inefficient during checkpoints; it involves first downloading the remote working state data to local memory and then uploading it to the checkpoint directory. Likewise, both restore and rescale face the similar problems due to superfluous data transmission.</span></li></ul><p class="ne-p"><span class="ne-text">In order to solve the above problems and improve checkpoint/restore/rescaling performance of disaggregated storage, this FLIP proposes: </span></p><ol class="ne-ol"><li><span class="ne-text">A new checkpoint strategy</span><span class="ne-text"> for disaggregated state storage: leverage RocksDB's low-level api to retain a consistent snapshot during the checkpoint synchronous phase; and then transfer the snapshot files to checkpoint directory during asynchronous phase;</span></li><li><span class="ne-text">Accelerating checkpoint/restore/rescaling by leverage fast-duplication of remote file system, which can bypass the local TaskManager when transferring data between remote working directory and checkpoint directory.</span></li></ol><h1 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-High-levelOverview"><span class="ne-text">High-level Overview</span></h1><p class="ne-p"><span class="ne-text">Within the disaggregated state storage, ForSt temporarily holds state data in memory to create immutable state files, subsequently flushing them to the distributed file system (working directory). The remote state files and the checkpoint files may reside in different file directories, but share the same underlying distribution file system. Based on this premise, checkpoint/restore/rescaling essentially involves data transfer between two directories (working directory and checkpoint directory) on one remote file system.</span></p><p class="ne-p"><span class="ne-text"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image confluence-external-resource image-center" draggable="false" alt="image.png" height="250" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1708588972830-9b965eb4-4a69-45bd-9455-fb27cb38c678.png?x-oss-process=image%2Fresize%2Cw_1238%2Climit_0" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1708588972830-9b965eb4-4a69-45bd-9455-fb27cb38c678.png?x-oss-process=image%2Fresize%2Cw_1238%2Climit_0"></span></span></p><p class="ne-p" style="text-align: left;"><strong><span class="ne-text">Checkpoint</span></strong><span class="ne-text">: During checkpointing, ForSt freezes the snapshot files collection to prevent them from deletion in the synchronous phase; then fast-duplicate these files to the checkpoint directory in asynchronous phase; finally unlock these files, allowing for their deletion.</span></p><p class="ne-p" style="text-align: left;"><span class="ne-text"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image confluence-external-resource image-center" draggable="false" alt="image.png" height="250" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1708588793465-bb55f93d-1471-478e-ae29-de0ae5204482.png?x-oss-process=image%2Fresize%2Cw_1296%2Climit_0" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1708588793465-bb55f93d-1471-478e-ae29-de0ae5204482.png?x-oss-process=image%2Fresize%2Cw_1296%2Climit_0"></span></span></p><p class="ne-p" style="text-align: left;"><strong><span class="ne-text">Restore & Rescale</span></strong><span class="ne-text">: Similarly, restore and rescaling can also employ fast-duplication to minimize network traffic and optimize recovery performance. Notice that since state files are not required to be downloaded to local storage, constraints on local disk space are no longer a concern for downscaling.</span></p><h1 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-ProposedChanges">Proposed Changes</h1><h2 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Checkpoint"><span class="ne-text">Checkpoint</span></h2><p class="ne-p"><span class="ne-text">The RocksDB (the foundation of ForSt) native core provides several ways to build a checkpoint for ForSt:</span></p><ul class="ne-ul"><li><strong><span class="ne-text">Option-1</span></strong><span class="ne-text"> </span><a class="external-link" href="https://rocksdb.org/blog/2015/11/10/use-checkpoints-for-efficient-snapshots.html" rel="nofollow"><span class="ne-text">Checkpoints</span></a><span class="ne-text"> feature[3] : </span><strong><span class="ne-text">Synchronously</span></strong><span class="ne-text"> hard-link all db files to a specified directory.</span></li><li><strong><span class="ne-text">Option-2 </span></strong><span class="ne-text"> </span><a class="external-link" href="https://rocksdb.org/blog/2014/03/27/how-to-backup-rocksdb.html" rel="nofollow"><span class="ne-text">BackupableDB</span></a><span class="ne-text"> feature [4]: </span><strong><span class="ne-text">Synchronously </span></strong><span class="ne-text">create another backup DB and hard-link/copy all files to the backup db directory. It supports replicating files across different filesystem with different 'Env'.</span></li><li><strong><span class="ne-text">Option-3</span></strong><span class="ne-text"> Low-level API combination: Use </span><em><span class="ne-text">GetLiveFiles()</span></em><span class="ne-text">, combining with </span><em><span class="ne-text">DisableFileDeletion() </span></em><span class="ne-text">and </span><em><span class="ne-text">EnableFileDeletion()</span></em><span class="ne-text"> to do DB physical copy [5]. The </span><em><span class="ne-text">GetLiveFiles() </span></em><span class="ne-text">can be used to capture a consistent snapshot of db files, and then these files could be duplicated asynchronously.</span></li></ul><p class="ne-p"><span class="ne-text">For Option-1, most remote filesystem don't support hard link and </span><span class="ne-text" style="color: rgb(51,51,51);">typically work around by </span><span class="ne-text" style="color: rgb(0,0,0);">duplication</span><span class="ne-text" style="color: rgb(51,51,51);"> API (eg. S3 copyObject[6], Azure Blob Storage copyBlob[7], Aliyun </span><span class="ne-text" style="color: rgb(23,43,77);">OSS </span><span class="ne-text" style="color: rgb(0,0,0);">copyObject</span><span class="ne-text" style="color: rgb(23,43,77);">[8]</span><span class="ne-text" style="color: rgb(51,51,51);">), which is significantly less efficient than hard-link of LocalFileSystem. Consequently, for the ForSt on DFS, the </span><strong><span class="ne-text" style="color: rgb(51,51,51);">synchronous</span></strong><span class="ne-text" style="color: rgb(51,51,51);"> execution of Option-1 will cause task thread to be blocked for an extended duration during snapshot synchronous phase, negatively impacting the task threads' processing performance.</span></p><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">Option-2 faces a dilemma similar to Option-1, as </span><strong><span class="ne-text" style="color: rgb(51,51,51);">synchronously</span></strong><span class="ne-text" style="color: rgb(51,51,51);"> copying or duplicating DB files results in extended blocking of task threads during snapshot synchronous phase. Moreover, in this way, the checkpointed SST files are managed by BackupableDB(TM side), which is different from the current way that the checkpoint files are managed by JM side .</span></p><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">In Option-3, </span><em><span class="ne-text">GetLiveFiles() </span></em><span class="ne-text">is a very lightweight operation which can be </span><span class="ne-text" style="color: rgb(51,51,51);">executed</span><span class="ne-text"> during snapshot </span><span class="ne-text" style="color: rgb(51,51,51);">synchronous phase to take a consistent snapshot; the io-intensive file duplication could be deferred to the asynchronous phase. Additionally, the management of checkpointed files (file create/delete/do incremental/...) can stay aligned with the existing mechanism.</span></p><p class="ne-p"><strong><span class="ne-text">Thus, we would like to choose Option-3 to support ForSt checkpointing, which </span></strong><strong><span class="ne-text" style="color: rgb(51,51,51);">offers greater flexibility and could integrate seamlessly with the existing checkpoint mechanisms.</span></strong></p><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">In comparing snapshot processes between remote ForSt (Option-3) and local RocksDBStateBackend, the main differences are:</span></p><ul class="ne-ul"><li><strong><span class="ne-text" style="color: rgb(51,51,51);">synchronous phase</span></strong><span class="ne-text" style="color: rgb(51,51,51);">: RocksdbStateBackend needs hard-link all db files into local snapshot dir; while ForSt employs </span><em><span class="ne-text" style="color: rgb(51,51,51);">GetLiveFiles() </span></em><span class="ne-text" style="color: rgb(51,51,51);">API to capture a consistent snapshot;</span></li><li><strong><span class="ne-text" style="color: rgb(51,51,51);">asynchronous phase</span></strong><span class="ne-text" style="color: rgb(51,51,51);">: RocksdbStateBackend uploads the local files to remote checkpoint-dir; while ForSt employs fast </span><span class="ne-text" style="color: rgb(0,0,0);">duplication</span><span class="ne-text" style="color: rgb(51,51,51);"> to replicate the remote db files to checkpoint-dir.</span></li></ul><h3 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Keystepsincheckpointsync&asyncphase"><span class="ne-text">Key steps in checkpoint sync & async phase</span></h3><ul class="ne-ul"><li><strong><span class="ne-text">synchronous phase</span></strong><span class="ne-text">:</span></li></ul><ol class="ne-list-wrap"><ol class="ne-ol"><li><span class="ne-text">Snapshot state MetaInfo; </span></li><li><span class="ne-text">DB.disableFileDeletions(); //Prevent rocksdb file deletions</span></li><li><span class="ne-text">LiveFiles liveFiles = DB.getLiveFiles(true); // capture a consistent snapshot and save the live // files Info within current version.</span></li></ol></ol><ul class="ne-ul"><li><strong><span class="ne-text">asynchronous phase</span></strong><span class="ne-text">:</span></li></ul><ol class="ne-list-wrap"><ol class="ne-ol"><li><span class="ne-text">Materialize the state MetaInfo;</span></li><li><span class="ne-text">Calculates the files collection to be duplicated based on the incremental/full policy;</span></li><li><span class="ne-text">Fast-duplicate/Copy the checkpoint files in parallel;</span></li><li><span class="ne-text">Generate KeyedStateHandle, and register it to JM;</span></li><li><span class="ne-text">db.enableFileDeletions(false); // allow rockdb to delete obsolete files</span></li></ol></ol><h3 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Filemanagement"><strong><span class="ne-text">File management</span></strong></h3><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">With disaggregated state storage, </span><span class="ne-text">two distinct sets of files are located within the remote file system:</span></p><ul class="ne-ul"><li><strong><span class="ne-text">Working Files</span></strong><span class="ne-text">: These are the live files actively utilized by ForSt during its runtime operations.</span></li><li><strong><span class="ne-text">Checkpoint Files</span></strong><span class="ne-text">: These files are employed to restore the system state to a particular moment in time.</span></li></ul><p class="ne-p"><span class="ne-text">It would be advantageous to reconsider and reorganize the file structure in the remote file system.</span></p><p class="ne-p"><strong><span class="ne-text">Remote directory layout</span></strong></p><p class="ne-p"><span class="ne-text">Considering that in current flink architecture, the checkpoint files/directory are owned by JobManager, and the working files/directory are owned by TaskManager, so the checkpoint directory and the ForSt's working directory will still use different directories for disaggregated state storage. This means that the remote state files still need to be copied/fast-duplicated to the checkpoint directory during checkpoint (fast-duplicate will be more lightweight), even through it is already on the DFS. In addition, to simplify user configuration, the default working directory can be a subdirectory of the checkpoint-taskowned directory without extra configuration. In this way, the remote directory layout of disaggregated state storage is shown as follows:</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">|--- Job-RemoteState-Checkpointing-Dir
|--- chk-xxx
|--- shared
|-- checkpoint files
|--- taskowned
|--- remote-state-working-dir
|--- subTask-state-sub-dir
|--- db
|--- working files</pre>
</div></div><h3 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Checkpointcomplete/subsume/abort"><span class="ne-text">Checkpoint complete/subsume/abort</span></h3><p class="ne-p"><span class="ne-text">These behavior of ForSt is the same as that of local RocksdbStateBackend. The StreamStateHandle(checkpointed files) which is generated by TM will be registered to JM, which manages the checkpoint files lifecycle as it does currently.</span></p><h2 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Restore"><span class="ne-text">Restore</span></h2><p class="ne-p"><span class="ne-text">The key steps of restoring an instance of ForSt:</span></p><ol class="ne-ol"><li><span class="ne-text">Rebuild LSM-tree of ForSt: Initialize the working directory as user configured; Fast Duplicate (Or Copy) all files from the corresponding subdir of restored checkpoint;</span></li><li><span class="ne-text">Subsume Last Checkpoint when first checkpoint completed: Works similar to previous policies in both claim and no-claim mode.</span></li></ol><h2 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Rescaling"><span class="ne-text">Rescaling</span></h2><p class="ne-p"><span class="ne-text">ForSt will also leverage ClipDB/IngestDB[9] to accelerate rescaling and utilize the fast-duplication capabilities of the remote filesystem to enhance the transfer speed of state files.</span><span class="ne-text" style="color: rgb(0,0,0);"> The key steps of ForSt rescaling are as follows:</span></p><ol class="ne-ol"><li><span class="ne-text">Each subTask fast-duplicate/copy the checkpoint files to its working dir, which may contains multiple ForSt intances;</span></li><li><span class="ne-text">Use ClipDB to clear invalid data of multiple ForSt instances based on KeyGroupRange;</span></li><li><span class="ne-text">Apply IngestDB to merge data from multiple ForSt instances.</span></li></ol><p class="ne-p" style="text-align: center;"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource image-center" draggable="false" height="250" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1708588899011-220a4f50-126b-4d93-9084-8966dcdd8423.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1708588899011-220a4f50-126b-4d93-9084-8966dcdd8423.png"></span></p><p class="ne-p"><span class="ne-text">Embedded RocksDB should download all DFS files to local disk before do clip or ingest, while remote ForSt can utilize fast-duplicate to speed up this process with the support of remote file system. Moreover, during rescaling of Flink job, which involves intensive I/O operations, fast-duplication offloads the I/O tasks to the remote file system, typically resulting in superior performance compared to file downloads.</span></p><h2 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Savepoint"><span class="ne-text">Savepoint</span></h2><ul class="ne-ul"><li><span class="ne-text">Native savepoint: follow the existing mechanism, that is, do a full checkpoint at another savepoint dir;</span></li><li><span class="ne-text">Canonical savepoint: follow the existing mechanism(build a full key-value iterator for ForSt);</span></li></ul><h2 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-RelatedFeaturesSupport"><span class="ne-text">Related Features Support</span></h2><ul class="ne-ul"><li><span class="ne-text">Unaligned checkpoint: The unaligned checkpoint behavior would not be affected by ForSt checkpoint strategy.</span></li><li><span class="ne-text">Changelog: The enhancements that the ChangelogStateBackend brings to checkpointing are akin to the improvements afforded by the disaggregated state storage. So I suggest disaggregated ForSt don't support ChangelogStateBackend.</span></li><li><span class="ne-text" style="color: rgb(51,51,51);">Local recovery: Within the context of disaggregated storage, the semantics of local recovery will be changed, which need to be combined with local-disk-cache. Future FLIPs will introduce the local disk cache to improve disaggregated storage performance. Subsequently, the local recovery feature could enhance the speed of local disk cache restoration.</span></li><li><span class="ne-text"><a href="https://cwiki.apache.org/confluence/x/DwsNDw" rel="nofollow">FLIP-306</a>[10]/Small File Merging Mechanism: the proposed basic checkpoint strategy would be compatiable with the small file merging mechanism.</span></li></ul><h2 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Mid/longtermfollowupwork"><span class="ne-text" style="color: rgb(23,43,77);">Mid/long term follow up work</span></h2><p class="ne-p"><strong><span class="ne-text">TM file ownership: </span></strong><span class="ne-text">Instead of letting JM manage the checkpoint files while TM manage the runtime state SSTs, TM takes the responsibility of managing all files considering both the lifecycles of checkpoints and live SSTs. Thus the DFS </span><strong><span class="ne-text">files can be easily reused when checkpointing and restoring.</span></strong><span class="ne-text"> No more file re-uploading or fast-copy should be processed during checkpoint or state recovery. This work may break some of flink's existing file management mechanisms, and we will continue to advance these work in future FLIPs.</span></p><h1 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Compatibility,Deprecation,andMigrationPlan">Compatibility, Deprecation, and Migration Plan</h1><p><span class="ne-text">he Checkpointing/Restoring/Rescaling stategies will be implemented based on the newly introduced ForSt, and follow the existing checkpoint/restore mechanism as far as possible. This will introduce some code refactor work for RocksdbStateBackend/Checkpointing/Restoring, but it won't break the compatibility of existing public interface (including RocksdbStateBackend public interface).</span></p><h1 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-TestPlan">Test Plan</h1><ul class="ne-ul"><li><span class="ne-text">New UT/ITs will be introduced for checkpointing/restoring/rescaling described in section 'Proposed Changes'. </span></li><li><span class="ne-text">New E2E tests of jobs that perform checkpointing/restoring/rescaling with ForSt will also be delivered.</span></li></ul><h1 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-RejectedAlternatives">Rejected Alternatives</h1><ul><li><strong><span class="ne-text">Use Rocksdb BackupableDB[4] to build a checkpoint for ForStStateBackend</span></strong></li></ul><p class="ne-p"><span class="ne-text"> Rocksdb BackupableDB feature offers a straightforward method to back up ForSt. It also support creating multiple backups which are incremental, and support deleting specified backups.</span></p><p class="ne-p"><strong><span class="ne-text">Rejected reason</span></strong><span class="ne-text">: 1) Flink must capture a consistent snapshot in synchronous phase of checkpointing, so the </span><em><span class="ne-text">BackupableDB.createNewBackup()</span></em><span class="ne-text"> must be executed in snapshot synchronous phase, which will block task thread for a long duration; 2) In this way, the checkpointed files are fully managed by BackupableDB(TM side), which is somewhat inconsistent with the current way that checkpoint files are managed by JM.</span></p><h1 id="FLIP428:FaultTolerance/RescaleIntegrationforDisaggregatedState-Reference"><span class="ne-text">Reference</span></h1><p class="ne-p"><span class="ne-text">[1] FLIP-423 <a class="" href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">https://cwiki.apache.org/confluence/x/R4p3EQ</a></span></p><p class="ne-p"><span class="ne-text">[2] FLIP-427 <a class="" href="https://cwiki.apache.org/confluence/x/T4p3EQ" rel="nofollow">https://cwiki.apache.org/confluence/x/T4p3EQ</a></span></p><p class="ne-p"><span class="ne-text">[3] </span><a class="external-link" href="https://rocksdb.org/blog/2015/11/10/use-checkpoints-for-efficient-snapshots.html" rel="nofollow"><span class="ne-text">https://rocksdb.org/blog/2015/11/10/use-checkpoints-for-efficient-snapshots.html</span></a></p><p class="ne-p"><span class="ne-text">[4] </span><a class="external-link" href="https://rocksdb.org/blog/2014/03/27/how-to-backup-rocksdb.html" rel="nofollow"><span class="ne-text">https://rocksdb.org/blog/2014/03/27/how-to-backup-rocksdb.html</span></a></p><p class="ne-p"><span class="ne-text">[5] </span><a class="external-link" href="https://github.com/facebook/rocksdb/wiki/Replication-Helpers#functions-for-full-db-physical-copy" rel="nofollow"><span class="ne-text">https://github.com/facebook/rocksdb/wiki/Replication-Helpers#functions-for-full-db-physical-copy</span></a></p><p class="ne-p"><span class="ne-text">[6] </span><a class="external-link" href="https://docs.aws.amazon.com/AmazonS3/latest/API/API_CopyObject.html" rel="nofollow"><span class="ne-text">https://docs.aws.amazon.com/AmazonS3/latest/API/API_CopyObject.html</span></a></p><p class="ne-p"><span class="ne-text">[7] </span><a class="external-link" href="https://learn.microsoft.com/en-us/rest/api/storageservices/copy-blob" rel="nofollow"><span class="ne-text">https://learn.microsoft.com/en-us/rest/api/storageservices/copy-blob</span></a></p><p class="ne-p"><span class="ne-text">[8] </span><a class="external-link" href="https://www.alibabacloud.com/help/en/oss/developer-reference/copyobject" rel="nofollow"><span class="ne-text">https://www.alibabacloud.com/help/en/oss/developer-reference/copyobject</span></a></p><p class="ne-p"><span class="ne-text">[9] </span><a class="external-link" href="https://github.com/facebook/rocksdb/wiki/Creating-and-Ingesting-SST-files#ingesting-sst-files" rel="nofollow"><span class="ne-text">https://github.com/facebook/rocksdb/wiki/Creating-and-Ingesting-SST-files#ingesting-sst-files</span></a></p><p class="ne-p"><span class="ne-text">[10] </span><a class="ne-link" href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-306%3A+Unified+File+Merging+Mechanism+for+Checkpoints" rel="nofollow"><span class="ne-text">https://cwiki.apache.org/confluence/display/FLINK/FLIP-306%3A+Unified+File+Merging+Mechanism+for+Checkpoints</span></a></p><p><br/></p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=293046865">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=293046865&revisedVersion=12&originalVersion=11">View Changes Online</a>
</div>
</div>Jinzhong Li2024-02-23T02:59:03ZFLIP-427: ForSt - Disaggregated state StoreHangxiang Yutag:cwiki.apache.org,2009:page-293046863-182024-03-28T01:49:20Z2024-02-23T02:57:53Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~masteryhx
">Hangxiang Yu</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<div class="table-wrap"><table class="wrapped confluenceTable" style="letter-spacing: 0.0px;"><thead><tr><th class="confluenceTh">Discussion thread</th><td class="confluenceTd"><p><a class="external-link" href="https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0" rel="nofollow">https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0</a></p><p><a class="external-link" href="https://lists.apache.org/thread/vktfzqvb7t4rltg7fdlsyd9sfdmrc4ft" rel="nofollow">https://lists.apache.org/thread/vktfzqvb7t4rltg7fdlsyd9sfdmrc4ft</a></p></td></tr></thead><colgroup><col/><col/></colgroup><tbody><tr><th colspan="1" class="confluenceTh">Vote thread</th><td colspan="1" class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/lqpovwx3kpszgfcgl0olfhxotfo9d6nz" rel="nofollow">https://lists.apache.org/thread/lqpovwx3kpszgfcgl0olfhxotfo9d6nz</a></td></tr><tr><th class="confluenceTh">JIRA</th><td class="confluenceTd"><br/></td></tr><tr><th colspan="1" class="confluenceTh">Release</th><td colspan="1" class="confluenceTd">2.0</td></tr></tbody></table></div><p>Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).</p><p><style type='text/css'>/*<![CDATA[*/
div.rbtoc1711724893212 {padding: 0px;}
div.rbtoc1711724893212 ul {margin-left: 0px;}
div.rbtoc1711724893212 li {margin-left: 0px;padding-left: 0px;}
/*]]>*/</style><div class='toc-macro rbtoc1711724893212'>
<ul class='toc-indentation'>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-Motivation'>Motivation</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-ProposedChanges'>Proposed Changes</a>
<ul class='toc-indentation'>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-ForStSide:RemoteFileSystemsIntergration'>ForSt Side : Remote File Systems Intergration</a>
<ul class='toc-indentation'>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-FSInterfacebridge'>FS Interface bridge</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-JNIRelated'>JNI Related</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-OtherbehaviorsinForSt'>Other behaviors in ForSt</a></li>
</ul>
</li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-FlinkSide:ImplementationDetails'>Flink Side: Implementation Details</a>
<ul class='toc-indentation'>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-ExtractCommonClasses'>Extract Common Classes</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-ImplementnewFileSystem'>Implement new FileSystem</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-LocalFileCache'>Local File Cache</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-Directorylayout'>Directory layout</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-Checkpointing&Recovery'>Checkpointing & Recovery</a></li>
</ul>
</li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-OtherRelatedImprovement'>Other Related Improvement</a></li>
</ul>
</li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-PublicInterfaces'>Public Interfaces</a>
<ul class='toc-indentation'>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-ForStStateBackendFactory'>ForStStateBackendFactory</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-ForStStateBackend'>ForStStateBackend</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-ForStOptions'>ForStOptions</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-Example'>Example</a></li>
</ul>
</li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-Limitations'>Limitations</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-RejectedAlternatives'>Rejected Alternatives</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-Compatibility,Deprecation,andMigrationPlan'>Compatibility, Deprecation, and Migration Plan</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-TestPlan'>Test Plan</a></li>
<li><a href='#FLIP427:ForStDisaggregatedstateStore-Reference'>Reference</a></li>
</ul>
</div></p><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p>This is a<span> </span><span style="color: rgb(23,43,77);">sub-FLIP for the disaggregated state management and its related work, please read the<span> </span><a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">FLIP-423</a><span> </span>first to know whole story.</span></p></div></div><p><br/></p><h1 id="FLIP427:ForStDisaggregatedstateStore-Motivation">Motivation</h1><p class="ne-p"><span class="ne-text">As described in FLIP-423, there are some tough issues about embedded state backend on local file system, respecially when dealing with extremely large state:</span></p><ol class="ne-ol"><li><strong><span class="ne-text" style="color: rgb(51,51,51);">Constraints of local disk space complicate the prediction of storage requirements, potentially leading to job failures</span></strong><span class="ne-text">: Especially in cloud native deployment mode, pre-allocated local disks typically face strict capacity constraints, making it challenging to forecast the size requirements of job states. Over-provisioning disk space results in unnecessary resource overhead, while under-provisioning risks job failure due to insufficient space.</span></li><li><strong><span class="ne-text">The tight coupling of compute and storage resources leads to underutilization and increased waste:</span></strong><span class="ne-text"> Jobs can generally be categorized as either CPU-intensive or IO-intensive. In a coupled architecture, CPU-intensive jobs leave a significant portion of storage resources underutilized, whereas IO-intensive jobs result in idle computing resources.</span></li></ol><p class="ne-p"><span class="ne-text">By considering remote storage as the primary storage, all working states are maintained on the remote file system, which brings several advantages:</span></p><ol class="ne-ol"><li><strong><span class="ne-text">Remote storages e.g. S3/HDFS typically offer elastic scalability, theoretically providing unlimited space.</span></strong></li><li><strong><span class="ne-text">The allocation of remote storage resources can be optimized by reducing them for CPU-intensive jobs and augmenting them for IO-intensive jobs, thus enhancing overall resource utilization.</span></strong></li><li><strong><span class="ne-text">This architecture facilitates a highly efficient and lightweight process for checkpointing, recovery, and rescaling through fast copy or simple move.</span></strong></li></ol><p class="ne-p"><span class="ne-text">This FLIP aims to realize disaggregated state for our new key-value store named </span><strong><span class="ne-text">ForSt</span></strong><span class="ne-text"> which evloves from RocksDB and supports remote file system. This makes Flink get rid of the disadvantages by coupled state architecture and embrace the scalable as well as flexible cloud-native storage.</span></p><h1 id="FLIP427:ForStDisaggregatedstateStore-ProposedChanges">Proposed Changes</h1><p class="ne-p"><span class="ne-text">As discussed in FLIP-423, RocksDB(frocksdb) is selected to support disaggregated storage architecture because it has been validated by extensive production practices with Flink jobs and it also boasts significant scalability.</span></p><p class="ne-p"><span class="ne-text">We plan to extend RocksDB to run on all file systems supported by Flink, thereby allowing state to be stored on various distributed file systems. The new proposed store is named </span><strong><span class="ne-text">ForSt</span></strong><span class="ne-text">, which stands for </span><strong><span class="ne-text">'</span></strong><strong><span class="ne-text" style="color: rgb(223,42,63);">For-St</span></strong><strong><span class="ne-text">reaming </span></strong><strong><span class="ne-text" style="color: rgb(223,42,63);">St</span></strong><strong><span class="ne-text">orage'</span></strong><span class="ne-text">. However, currently RocksDB only supports local file systems, so additional efforts are required to support this:</span></p><ol class="ne-ol"><li><strong><span class="ne-text">Remote File Systems Intergration in </span></strong><em><strong><span class="ne-text" style="text-decoration: underline;">ForSt side</span></strong></em><strong><span class="ne-text">.</span></strong><span class="ne-text"> We will leverage JNI bridge to proxy any I/O operations from ForSt to Flink FileSystem interface, where many file systems have already been supported.</span></li><li><strong><span class="ne-text">Implement new State Backend in </span></strong><em><strong><span class="ne-text" style="text-decoration: underline;">Flink side</span></strong></em><strong><span class="ne-text">. </span></strong><span class="ne-text">We will introduce a new state backend called ForStStateBackend to support ForSt.</span></li></ol><h2 id="FLIP427:ForStDisaggregatedstateStore-ForStSide:RemoteFileSystemsIntergration"><span class="ne-text">ForSt Side : Remote File Systems Intergration</span></h2><p><span class="confluence-embedded-file-wrapper"><img class="confluence-embedded-image confluence-external-resource" draggable="false" alt="image.png" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/310795/1708507562848-c5291fad-f934-48de-a19c-d95efb631047.png?x-oss-process=image%2Fresize%2Cw_1500%2Climit_0" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/310795/1708507562848-c5291fad-f934-48de-a19c-d95efb631047.png?x-oss-process=image%2Fresize%2Cw_1500%2Climit_0"></span></p><p class="ne-p"><span class="ne-text">ForSt is built on top of RocksDB. While RocksDB supports pluggable filesystem[1] which helps users to build RocksDB on different filesystems conveniently, ForSt should integrate remote filesystems . There are two proposals implementing this as we could see in the above graph:</span></p><ul class="ne-ul"><li><span class="ne-text">Proposal 1: Implement every file system based on ForSt's filesystem interface.</span></li><li><strong><span class="ne-text">Proposal 2: Leverage JNI bridge to proxy any I/O operations from ForSt to Flink FileSystem interface, where many file systems have already been supported.</span></strong></li></ul><div class="table-wrap"><table class="wrapped confluenceTable"><tbody><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Compared Proposal 1 to Proposal 2</span></strong></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Performace</span></strong></p><p class="ne-p"><strong><span class="ne-text">(Proposal 1 is better)</span></strong></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Proposal 2 has additional JNI overhead, but the overhead is relatively negligible when weighed against the latency of remote I/O.</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Achievability</span></strong></p><p class="ne-p"><strong><span class="ne-text">(Proposal 2 is better)</span></strong></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Due to the lack of a unified abstraction for different file systems, Proposal 1 requires implementation in C++ for each file system, which constitutes a considerable amount of work.</span></p><p class="ne-p"><span class="ne-text">While for Proposal 2, the JNI bridge shoud be established once between ForSt's interfaces and Flink's. It is a one-time job that works for every file systems that Flink has already supported.</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Maintainability</span></strong></p><p class="ne-p"><strong><span class="ne-text">(Proposal 2 is better)</span></strong></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">For Proposal 1, all different versions and dependencies related to the C++ file system libraries must be independently maintained and kept consistent with Flink.</span></p><p class="ne-p"><span class="ne-text">For Proposal 2, the critical logic of JNI bridge is consistent. The detailed implementations are maintained in Flink, which is much more easier for further developing.</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Scalability</span></strong></p><p class="ne-p"><strong><span class="ne-text">(Proposal 2 is better)</span></strong></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">For Proposal 2, the implementation in ForSt universal, all filesystems supported in Flink can be supported easily in java.</span></p></td></tr></tbody></table></div><p class="ne-p"><strong><span class="ne-text">As compared in the table, we'd like to choose proposal 2 firstly to support commonly used filesystems which Flink supports, e.g. hdfs, s3 and oss by using JNI to bridge the gap between ForSt FileSystem and Flink FileSystem.</span></strong></p><p class="ne-p"><strong><span class="ne-text">Of course, proposal 1 could be considered as an optimization to improve the performance in the future. This could be implemented in ForSt side, then Flink side only need to change used env[6] to support it.</span></strong></p><p class="ne-p"><span class="ne-text">We will introduce some key points of the intergration for proposal 2 in the following subsections.</span></p><h3 id="FLIP427:ForStDisaggregatedstateStore-FSInterfacebridge"><span class="ne-text">FS Interface bridge</span></h3><p class="ne-p"><span class="ne-text">As outlined above, this FLIP employs a </span><strong><span class="ne-text">JNI bridge</span></strong><span class="ne-text"> to redirect all I/O operations from ForSt to the Flink FileSystem interface. The table below enumerates the interfaces from ForSt and specifies the corresponding Flink FileSystem interfaces that the implementation should use.</span></p><div class="table-wrap"><table class="wrapped confluenceTable"><tbody><tr><td colspan="3" class="confluenceTd"><p class="ne-p" style="text-align: center;"><strong><span class="ne-text">Interface Reference</span></strong></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Interface of ForSt</span></strong></p></td><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Which interface of Flink should be used </span></strong></p></td><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Comment</span></strong></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Constructor</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">get</span></p></td><td class="confluenceTd"><ol class="ne-ol"><li><span class="ne-text">Flink FileSystem#initialize should be called before this so that all filesystems are usable in Flink side</span></li><li><span class="ne-text">ForSt maintains the life cycle of filesystem object </span></li></ol></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">FileExists</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">exists</span></p></td><td rowspan="11" class="confluenceTd"><p class="ne-p" style="text-align: left;"><span class="ne-text">Could be implemented using the method of Flink simply</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">CreateDir</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">mkdirs</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">CreateDirIfMissing</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">mkdirs</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">DeleteDir</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">delete</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">DeleteFile</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">delete</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">GetChildren</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">listStatus</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">RenameFile</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">rename</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">GetFileSize</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">getFileStatus</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">GetFileModificationTime</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">getFileStatus</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">IsDirectory</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">getFileStatus</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">NewDirectory -> FSDirectory</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">getFileStatus</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">LockFile/UnlockFile</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">-</span></p></td><td class="confluenceTd"><p class="ne-p" style="text-align: left;"><span class="ne-text">Fine if unsupported: Lock for different DB processors, Flink doesn't rely on this.</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">NewSequentialFile -></span></p><p class="ne-p"><span class="ne-text">FSSequentialFile</span></p></td><td rowspan="2" class="confluenceTd"><p class="ne-p"><span class="ne-text">open -> FSDataInputStream</span></p></td><td rowspan="3" class="confluenceTd"><ol class="ne-ol"><li><span class="ne-text">ForSt maintains the life cycle of FSDataOutputStream/FSDataInputStream</span></li><li><span class="ne-text">Flink FSDataOutputStream/FSDataInputStream should support ByteBuffer to read and write to avoid copying data frequently</span></li></ol></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">NewRandomAccessFile -> FSRandomAccessFile</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">NewWritableFile -> FSWritableFile</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">create -> FSDataOutputStream</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">NewLogger -> Logger</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">-</span></p></td><td class="confluenceTd"><p class="ne-p" style="text-align: left;"><span class="ne-text">LOG in FileSystem,Could be implemented by NewWritableFile</span></p></td></tr></tbody></table></div><h3 id="FLIP427:ForStDisaggregatedstateStore-JNIRelated"><span class="ne-text">JNI Related</span></h3><p class="ne-p"><span class="ne-text">JNI is a part of the Java platform that allows Java code to interact with code written in other languages, typically C or C++. JNIEnv is the key component of JNI through which native methods can interact with the JVM, providing rich methods e.g. creating new Java Objects, calling methods, handling exceptions. To interact with JVM in C++ side, three steps should be followed:</span></p><div class="table-wrap"><table class="wrapped confluenceTable"><tbody><tr><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Step</span></strong></p></td><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">The core issue to be resolved</span></strong></p></td><td class="confluenceTd"><p class="ne-p"><strong><span class="ne-text">Solution</span></strong></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Step 1: Init and maintain JNIEnv</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">The JNIEnv each call used should be consistent with the one used by the JVM of TaskManager </span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">Global JNIEnv</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Step 2: </span><span class="ne-text" style="color: rgb(51,51,51);">Use JNIEnv to create Java Objects and call corresponding methods</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">current methods about reading/ writing files of Flink FileSystem are not enough due to unavoidable data copy between C++ memory and Java heap as we could see in the above table.</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">Reading/Writing files with ByteBuffer</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Step 3: </span><span class="ne-text" style="color: rgb(51,51,51);">Maintain the creation and release of these objects </span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">Since the ForSt is the one that calls the interfaces, the objects' lifecycle should be managed in DB side.</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">Maintain the lifecycle of these objects </span></p></td></tr></tbody></table></div><p class="ne-p"><span class="ne-text">The details of the three solutions are as follows:</span></p><ol class="ne-ol"><li><strong><span class="ne-text">Global JNIEnv. </span></strong><span class="ne-text">The JNIEnv could be maintained globally by implementing JNI_Onload[2] and AttachCurrentThread[3]. Then All JNI call will use JNIEnv same as the JVM of TaskManager.</span></li><li><strong><span class="ne-text" style="color: rgb(51,51,51);">Reading/Writing files with ByteBuffer. </span></strong><span class="ne-text" style="color: rgb(51,51,51);">We need to introduce a </span><strong><span class="ne-text" style="color: rgb(51,51,51);">ForStFileSystem</span></strong><span class="ne-text" style="color: rgb(51,51,51);"> in Flink to support reading and writing data by ByteBuffer whose pseudocode could be seen in "Implement new FileSystem". Then ForSt could allocate ByteBuffer and use it to call new methods of this class. The lifecycle of ByteBuffer is maintained by ForSt as described below.</span></li><li><strong><span class="ne-text" style="color: rgb(51,51,51);">Maintain the lifecycle of these objects. </span></strong><span class="ne-text" style="color: rgb(51,51,51);">The lifecycle of all objects from Flink FileSystem, e.g. FileSystem, FSDataOutputStream, FSDataInputStream should be maintained in ForSt. </span><span class="ne-text">As we could see in the below graph, The lifecycle of these objects:</span></li></ol><ol class="ne-list-wrap"><ol class="ne-ol"><li><span class="ne-text" style="color: rgb(51,51,51);">ForSt creates some objects by methods of FileSystem in Flink side when ForSt accesses its FileSystem. these objects are saved as a member variable in ForSt side.</span></li><li><span class="ne-text" style="color: rgb(51,51,51);">ForSt will use these objects to executes some FileSystem operations including reading and writing files at runtime.</span></li><li><span class="ne-text" style="color: rgb(51,51,51);">These objects are destructed when these classes are released.</span></li></ol></ol><p><span class="ne-text" style="color: rgb(51,51,51);"><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image confluence-external-resource" draggable="false" alt="image.png" height="400" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/310795/1708507597522-48b9f9e1-c46a-4c6d-ad3e-2b4f582fa4d8.png?x-oss-process=image%2Fresize%2Cw_1016%2Climit_0" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/310795/1708507597522-48b9f9e1-c46a-4c6d-ad3e-2b4f582fa4d8.png?x-oss-process=image%2Fresize%2Cw_1016%2Climit_0"></span> <span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image" draggable="false" height="400" src="https://cwiki.apache.org/confluence/download/attachments/293046863/image-2024-2-28_18-13-34.png?version=1&modificationDate=1709115215000&api=v2" data-image-src="https://cwiki.apache.org/confluence/download/attachments/293046863/image-2024-2-28_18-13-34.png?version=1&modificationDate=1709115215000&api=v2" data-unresolved-comment-count="0" data-linked-resource-id="293047183" data-linked-resource-version="1" data-linked-resource-type="attachment" data-linked-resource-default-alias="image-2024-2-28_18-13-34.png" data-base-url="https://cwiki.apache.org/confluence" data-linked-resource-content-type="image/png" data-linked-resource-container-id="293046863" data-linked-resource-container-version="18" alt=""></span></span></p><h3 id="FLIP427:ForStDisaggregatedstateStore-OtherbehaviorsinForSt"><span class="ne-text">Other behaviors in ForSt</span></h3><p class="ne-p"><span class="ne-text">The other parts of ForSt built on the file system, such as reading, writing and compation, all those parts are similar as local. Some stratiges could be introduced to improve which will be described in "Future Avaliable Improvement".</span></p><h2 id="FLIP427:ForStDisaggregatedstateStore-FlinkSide:ImplementationDetails"><span class="ne-text">Flink Side: Implementation Details </span></h2><p class="ne-p"><span class="ne-text">As discussed in FLIP-423, we will introduce a new state backend called </span><strong><span class="ne-text">ForStStateBackend</span></strong><span class="ne-text"> to support the ForSt.</span></p><p class="ne-p"><span class="ne-text">Basically, many classes could be extracted and reused (see "Extract Common Classes"). </span><strong><span class="ne-text">Remained classes containing the logic of </span></strong><strong><span class="ne-text" style="color: rgb(34,34,34);">Initialization</span></strong><strong><span class="ne-text">, Configuration, Checkpointing, Restoring should be re-implemeted</span></strong><span class="ne-text"> e.g. ForStStateBackendFactory / ForStStateBackend / ForStKeyedStateBackend: </span></p><ol class="ne-ol"><li><span class="ne-text" style="color: rgb(34,34,34);">Initialization</span><span class="ne-text">: The directory should be </span><span class="ne-text" style="color: rgb(34,34,34);">initialized</span><span class="ne-text"> in the remote storage</span></li><li><span class="ne-text">Configuration: some extra configurations should be </span><span class="ne-text" style="color: rgb(34,34,34);">parsed</span><span class="ne-text"> and some unsupported configurations should be checked</span></li><li><span class="ne-text">Checkpoint & Restoring & Rescaling: The strategy of checkpointing & restoring & rescaling is different from ForStStateBackend due to existing files in remote storage which will be described in "Basic Strategy of Checkpointing & Restoring & Rescaling"</span></li></ol><p class="ne-p"><span class="ne-text">Besides, As we discussed in "Methods of reading/writing files", we also need to implement a </span><strong><span class="ne-text">ForStFileSystem</span></strong><span class="ne-text"> to support reading and writing with ByteBuffer which will be introduced in "Implement ForStFileSystem".</span></p><h3 id="FLIP427:ForStDisaggregatedstateStore-ExtractCommonClasses"><span class="ne-text">Extract Common Classes</span></h3><p class="ne-p"><span class="ne-text">Many classes are common for RocksDBStateBackend and ForStStateBackend. For example :</span></p><ol class="ne-ol"><li><span class="ne-text">related to basic configuration. e.g. RocksDBOptions, PredefinedOptions</span></li><li><span class="ne-text">related to resource control. e.g. RocksDBResourceContainer, RocksDBSharedResources</span></li><li><span class="ne-text">related to state interface. e.g. RocksDBValueState, RocksDBMapState</span></li></ol><p class="ne-p"><span class="ne-text">So we'd like to </span><strong><span class="ne-text">extract a common Module called "flink-statebackend-rocksdb-common" and put these common classes into the new module</span></strong><span class="ne-text">.</span></p><h3 id="FLIP427:ForStDisaggregatedstateStore-ImplementnewFileSystem"><span class="ne-text">Implement new FileSystem</span></h3><p class="ne-p"><span class="ne-text">A new class called </span><strong><span class="ne-text">ForStFileSystem</span></strong><span class="ne-text"> which wrappers FileSystem will be introduced to be responsible for:</span></p><ol class="ne-ol"><li><strong><span class="ne-text">Supporting reading/writing data from/to remote storage with ByteBuffer</span></strong></li><li><span class="ne-text">Introducing some metrics in filesystem level to monitor reading and </span><span class="ne-text" style="color: rgb(34,34,34);">writing</span></li><li><span class="ne-text">Introducing cache strategies in filesystem level</span></li></ol><p class="ne-p"><span class="ne-text">Its pseudocode could be seen as below.</span></p><p class="ne-p"><br/></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>ForStFileSystem.java</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">/**
* ForStFileSystem is a class used to wrapper current flink FileSystem
* and expose some interfaces to the ForSt .
*/
public class ForStFileSystem extends FileSystem {
// original FileSystem
private final FileSystem flinkFS;
public static FileSystem get(URI uri) throws IOException {
// Wrapper original FileSystem#get(URI uri)
}
// Wrapper other methods in FileSystem
@Override
public ByteBufferReadableFSDataInputStream open(Path f, int bufferSize) throws IOException {
// Call the original FileSystem#open
FSDataInputStream original = flinkFS.open(f, bufferSize);
long fileSize = flinkFS.getFileStatus(f).getLen();
// Wrapper with ByteBufferReadableFSDataInputStream
return new ByteBufferReadableFSDataInputStream(f, original,
() -> flinkFS.open(f, bufferSize),
fileSize);
}
@Override
public ByteBufferWritableFSDataOutputStream create(Path f, WriteMode overwriteMode) throws IOException {
// Call the original FileSystem#create
FSDataOutputStream original = flinkFS.create(f, overwriteMode);
// Wrapper with ByteBufferWritableFSDataOutputStream
return new ByteBufferWritableFSDataOutputStream(f, original);
}</pre>
</div></div><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>ByteBufferReadableFSDataInputStream.java</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">public class ByteBufferReadableFSDataInputStream extends FSDataInputStream {
public ByteBufferReadableFSDataInputStream(
Path path,
FSDataInputStream fsdis,
long totalFileSize) {
// init
}
public int readFully(long position, ByteBuffer bb) throws IOException {
// Read data from remote storage with ByteBuffer which will be called in the ForSt
}
// Wrapper other methods in FSDataInputStream
}</pre>
</div></div><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>ByteBufferWritableFSDataOutputStream.java</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">public class ByteBufferWritableFSDataOutputStream extends FSDataOutputStream {
public ByteBufferWritableFSDataOutputStream(Path path,
FSDataOutputStream fsdos) {
// init
}
public void write(ByteBuffer bb) throws IOException {
// Write data to remote storage with ByteBuffer which will be called in the ForSt
}
// Wrapper other methods in FSDataOutputStream
}</pre>
</div></div><p class="ne-p"><span class="ne-text">ForSt will init </span><code class="ne-code"><span class="ne-text">ForStFileSystem</span></code><span class="ne-text"> and call its </span><code class="ne-code"><span class="ne-text">open</span></code><span class="ne-text"> and </span><code class="ne-code"><span class="ne-text">create</span></code><span class="ne-text"> methods by JNI, then use </span><code class="ne-code"><span class="ne-text">int readFully(long position, ByteBuffer bb)</span></code><span class="ne-text"> and </span><code class="ne-code"><span class="ne-text">void write(ByteBuffer bb)</span></code><span class="ne-text"> to read or write data.</span></p><h3 id="FLIP427:ForStDisaggregatedstateStore-LocalFileCache"><span class="ne-text">Local File Cache</span></h3><p class="ne-p"><span class="ne-text">Always reading and writing files from remote storage can lead to a significant increase in latency. Therefore, we introduced local file cache to speed up file I/O operations.</span></p><p class="ne-p"><span class="ne-text">We could leverage existing </span><code class="ne-code"><span class="ne-text">LocalFileSystem</span></code><span class="ne-text"> and introduce a FIFO File Cache mechanism to make full use of local disk space, prioritizing retrieval from the </span><code class="ne-code"><span class="ne-text">LocalFileSystem</span></code><span class="ne-text">. Only when local disk space is insufficient, </span><code class="ne-code"><span class="ne-text">ForStFileSystem</span></code><span class="ne-text"> will be accessed.</span></p><h3 id="FLIP427:ForStDisaggregatedstateStore-Directorylayout"><span class="ne-text">Directory layout</span></h3><p class="ne-p"><strong><span class="ne-text">Working files </span></strong><span class="ne-text">are the live files actively utilized by ForSt during its runtime operations which are located within the remote file system.</span></p><p class="ne-p"><span class="ne-text">The directory layout of working files is shown as below. It's maintained in the directory of "Job-checkpointing-dir/taskowned" by default, whose directory could be modified by the new option which is described in "Public Interface Changes". The life cycle of working dir is managed as before local strategy. </span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>The directory layout of working set</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">// Default layout by default
|--- Job-checkpointing-dir
|--- chk-xxx
|--- shared
|--- taskowned
| --- working file dir
|--- subTask-sub-dir
|--- db
| --- working files
// User-defined directory
|--- User-defined-RemoteState-working-dir
|--- subTask-sub-dir
|--- db
| --- working files</pre>
</div></div><h3 id="FLIP427:ForStDisaggregatedstateStore-Checkpointing&Recovery"><span class="ne-text">Checkpointing & Recovery</span></h3><p class="ne-p"><span class="ne-text">The checkpointing and recovery involve copying files from one location to another. For the very basic version of ForStStateBackend, it will do downloading and re-uploading to copy files within remote file systems. This is pretty much like the behavior of RocksDBStateBackend except that the hard-link is unavailable for DFS.</span></p><p class="ne-p"><span class="ne-text">The FLIP-428 proposes a more lightweight checkpointing and recovery, which leverages file reusing and fast-copy mechanism. Please refer to FLIP-428 for more details.</span></p><h2 id="FLIP427:ForStDisaggregatedstateStore-OtherRelatedImprovement"><span class="ne-text">Other Related Improvement</span></h2><p class="ne-p"><span class="ne-text">There are many </span><span class="ne-text" style="color: rgb(34,34,34);">available</span><span class="ne-text"> improvement will be described in other FLIPs:</span></p><ol class="ne-ol"><li><span class="ne-text">Async IO: Async IO could help a lot </span><span class="ne-text" style="color: rgb(34,34,34);">especially</span><span class="ne-text"> for IO-intensive jobs, see FLIP-424 & FLIP-425 for more details.</span></li><li><span class="ne-text">Batch MultiGet: RocksDB multiGet[4] could reduce IO </span><span class="ne-text" style="color: rgb(34,34,34);">requests</span><span class="ne-text"> greatly, which will be introduced in the future.</span></li><li><span class="ne-text">Cache: The performance could be similar or even better than ForStStateBackend if all cache hits (after supporting Async IO and MultiGet), see FLIP-429 for more details.</span></li><li><span class="ne-text">Remote Compaction: Local Compation may cause a waste of resources. We could leverage remote compaction[5] to improve this, see FLIP-430 for more details.</span></li></ol><h1 id="FLIP427:ForStDisaggregatedstateStore-PublicInterfaces">Public Interfaces</h1><h2 id="FLIP427:ForStDisaggregatedstateStore-ForStStateBackendFactory"><span class="ne-text">ForStStateBackendFactory</span></h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>ForStStateBackendFactory.java</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">/** A factory that creates an {@link ForStStateBackend} from a configuration. */
@PublicEvolving
public class ForStStateBackendFactory
implements StateBackendFactory<ForStStateBackend> {
@Override
public ForStStateBackend createFromConfig(ReadableConfig config, ClassLoader classLoader)
throws IllegalConfigurationException {
return new ForStStateBackend().configure(config, classLoader);
}
}</pre>
</div></div><h2 id="FLIP427:ForStDisaggregatedstateStore-ForStStateBackend"><span class="ne-text">ForStStateBackend</span></h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>ForStStateBackend.java</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">/** A StateBackend that uses ForSt to store remote state. */
@PublicEvolving
public class ForStStateBackend extends AbstractManagedMemoryStateBackend implements ConfigurableStateBackend {
/**
* Creates a copy of this state backend that uses the values defined in the configuration for
* fields where that were not yet specified in this state backend.
*
* @param config The configuration.
* @param classLoader The class loader.
* @return The re-configured variant of the state backend
*/
@Override
public ForStStateBackend configure(ReadableConfig config, ClassLoader classLoader) {
return new ForStStateBackend(this, config, classLoader);
}
// Implement other methods of AbstractManagedMemoryStateBackend
}</pre>
</div></div><h2 id="FLIP427:ForStDisaggregatedstateStore-ForStOptions"><span class="ne-text">ForStOptions</span></h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>ForStOptions.java</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">/** Configuration options for the ForSt backend. */
@PublicEvolving
public class ForStOptions {
public static final ConfigOption<String> FORST_WORKING_DIR =
ConfigOptions.key("state.backend.forSt.working-dir")
.stringType()
.noDefaultValue()
.withDescription("The base working directory of forSt state backend. Support all file system schemas which flink uses, e.g. hdfs, s3, oss. The libraries of related file systems must be included. It will be generated automatically in 'Job-checkpointing-dir/taskowned' if no configured");
}</pre>
</div></div><h2 id="FLIP427:ForStDisaggregatedstateStore-Example"><span class="ne-text">Example</span></h2><p class="ne-p"><span class="ne-text">After configuring </span><code class="ne-code"><span class="ne-text">state.backend</span></code><span class="ne-text"> and </span><code class="ne-code"><span class="ne-text">state.backend.forSt.working-dir</span></code><span class="ne-text">, The job will use </span><code class="ne-code"><span class="ne-text">ForStStateBackend</span></code><span class="ne-text"> as the new State Backend. All working files will be stored on the remote file systems.</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>flink-conf.yaml</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">// Configure the new State Backend
state.backend.type: org.apache.flink.state.forSt.ForStStateBackendFactory
// Configure the working directory of remote state, e.g. {hdfs/s3/oss}://remote-state-dir
state.backend.forSt.working-dir: hdfs://remote-state-dir</pre>
</div></div><h1 id="FLIP427:ForStDisaggregatedstateStore-Limitations"><span class="ne-text">Limitations</span></h1><ol class="ne-ol"><li><span class="ne-text">Only support FileSystems which Flink supports.</span></li><li><span class="ne-text">The file system of working is same as the one used by checkpoint.</span></li></ol><h1 id="FLIP427:ForStDisaggregatedstateStore-RejectedAlternatives"><span class="ne-text">Rejected Alternatives</span></h1><p class="ne-p"><span class="ne-text">Rejected Proposal 1: Implement all filesystems using their C++ library</span></p><p class="ne-p"><span class="ne-text">Rejected Reason: Could be seen in "Supports All FileSystems which Flink supports"</span></p><h1 id="FLIP427:ForStDisaggregatedstateStore-Compatibility,Deprecation,andMigrationPlan"><span class="ne-text" style="color: rgb(23,43,77);">Compatibility, Deprecation, and Migration Plan</span></h1><p class="ne-p"><span class="ne-text">In theory, it could be compatible with RocksDBStateBackend even if we use different restoring and checkpointing strategy.</span></p><h1 id="FLIP427:ForStDisaggregatedstateStore-TestPlan"><span class="ne-text" style="color: rgb(23,43,77);">Test Plan</span></h1><p class="ne-p"><span class="ne-text">Tests contains:</span></p><ol class="ne-ol"><li><span class="ne-text">Existing UTs/ITs of ForStStateBackend</span></li><li><span class="ne-text">New UT about FlinkFileSystem in new ForSt</span></li><li><span class="ne-text">New UT about ForStFileSystem in Flink</span></li></ol><h1 id="FLIP427:ForStDisaggregatedstateStore-Reference"><span class="ne-text">Reference</span></h1><p class="ne-p"><span class="ne-text">[1] </span><a class="external-link" href="https://github.com/facebook/rocksdb/blob/main/include/rocksdb/file_system.h" rel="nofollow"><span class="ne-text">https://github.com/facebook/rocksdb/blob/main/include/rocksdb/file_system.h</span></a></p><p class="ne-p"><span class="ne-text">[2] </span><a class="external-link" href="https://docs.oracle.com/javase/8/docs/technotes/guides/jni/spec/invocation.html#JNJI_OnLoad" rel="nofollow"><span class="ne-text">https://docs.oracle.com/javase/8/docs/technotes/guides/jni/spec/invocation.html#JNJI_OnLoad</span></a></p><p class="ne-p"><span class="ne-text">[3] </span><a class="external-link" href="https://docs.oracle.com/javase/8/docs/technotes/guides/jni/spec/invocation.html#AttachCurrentThread" rel="nofollow"><span class="ne-text">https://docs.oracle.com/javase/8/docs/technotes/guides/jni/spec/invocation.html#AttachCurrentThread</span></a></p><p class="ne-p"><span class="ne-text">[4] </span><a class="external-link" href="https://rocksdb.org/blog/2022/10/07/asynchronous-io-in-rocksdb.html" rel="nofollow"><span class="ne-text">https://rocksdb.org/blog/2022/10/07/asynchronous-io-in-rocksdb.html</span></a></p><p class="ne-p"><span class="ne-text">[5] </span><a class="external-link" href="https://github.com/facebook/rocksdb/wiki/Remote-Compaction-(Experimental)" rel="nofollow"><span class="ne-text">https://github.com/facebook/rocksdb/wiki/Remote-Compaction-(Experimental)</span></a></p><p class="ne-p"><span class="ne-text">[6] <a class="external-link" href="https://github.com/facebook/rocksdb/blob/main/include/rocksdb/env.h" rel="nofollow">https://github.com/facebook/rocksdb/blob/main/include/rocksdb/env.h</a></span></p><p><br/></p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-427%3A+ForSt+-+Disaggregated+state+Store">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=293046863&revisedVersion=18&originalVersion=17">View Changes Online</a>
</div>
</div>Hangxiang Yu2024-02-23T02:57:53ZFLIP-426: Grouping Remote State AccessJinzhong Litag:cwiki.apache.org,2009:page-293046861-202024-03-27T12:15:19Z2024-02-23T02:55:45Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~lijinzhong
">Jinzhong Li</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<div class="table-wrap"><table class="wrapped confluenceTable" style="letter-spacing: 0.0px;"><thead class=""><tr class=""><th class="confluenceTh">Discussion thread</th><td class="confluenceTd"><p><a class="external-link" href="https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0" rel="nofollow">https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0</a></p><p><a class="external-link" href="https://lists.apache.org/thread/bt931focfl9971cwq194trmf3pkdsxrf" rel="nofollow">https://lists.apache.org/thread/bt931focfl9971cwq194trmf3pkdsxrf</a></p></td></tr></thead><colgroup class=""><col class=""/><col class=""/></colgroup><tbody class=""><tr class=""><th colspan="1" class="confluenceTh">Vote thread</th><td colspan="1" class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/0yvspwhsf0vyvqkkd7snzg33k95v3jbc" rel="nofollow">https://lists.apache.org/thread/0yvspwhsf0vyvqkkd7snzg33k95v3jbc</a></td></tr><tr class=""><th class="confluenceTh">JIRA</th><td class="confluenceTd"><br/></td></tr><tr class=""><th colspan="1" class="confluenceTh">Release</th><td colspan="1" class="confluenceTd">2.0</td></tr></tbody></table></div><p>Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).</p><p><style type='text/css'>/*<![CDATA[*/
div.rbtoc1711724893234 {padding: 0px;}
div.rbtoc1711724893234 ul {margin-left: 0px;}
div.rbtoc1711724893234 li {margin-left: 0px;padding-left: 0px;}
/*]]>*/</style><div class='toc-macro rbtoc1711724893234'>
<ul class='toc-indentation'>
<li><a href='#FLIP426:GroupingRemoteStateAccess-Motivation'>Motivation</a></li>
<li><a href='#FLIP426:GroupingRemoteStateAccess-ProposedChanges'>Proposed Changes</a>
<ul class='toc-indentation'>
<li><a href='#FLIP426:GroupingRemoteStateAccess-DataStructureofBatchStateRequests'>Data Structure of Batch State Requests</a></li>
<li><a href='#FLIP426:GroupingRemoteStateAccess-CategorizeandProcess'>Categorize and Process</a></li>
<li><a href='#FLIP426:GroupingRemoteStateAccess-LeverageRocksDBMultiGet'>Leverage RocksDB MultiGet</a></li>
<li><a href='#FLIP426:GroupingRemoteStateAccess-CodeExampleonHowtoAccessStateUsingBatch'>Code Example on How to Access State Using Batch</a></li>
</ul>
</li>
<li><a href='#FLIP426:GroupingRemoteStateAccess-PublicInterfaces'>Public Interfaces</a></li>
<li><a href='#FLIP426:GroupingRemoteStateAccess-Compatibility,Deprecation,andMigrationPlan'>Compatibility, Deprecation, and Migration Plan</a></li>
<li><a href='#FLIP426:GroupingRemoteStateAccess-TestPlan'>Test Plan</a></li>
<li><a href='#FLIP426:GroupingRemoteStateAccess-RejectedAlternatives'>Rejected Alternatives</a></li>
</ul>
</div></p><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p><span style="color: rgb(51,51,51);">This is a<span> </span></span><span style="color: rgb(23,43,77);">sub-FLIP for the disaggregated state management and its related work, please read the<span> </span><a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">FLIP-423</a><span> </span>first to know the whole story.</span></p></div></div><h1 id="FLIP426:GroupingRemoteStateAccess-Motivation">Motivation</h1><p class="ne-p"><span class="ne-text">I/O speed and latency are critical for overall data throughput, particularly in jobs that manage large states. Implementing multiple asynchronous I/O operations is a proven strategy to enhance throughput by increasing parallelism of I/O execution. However, simply expanding I/O parallelism can quickly hit a ceiling due to finite I/O bandwidth. Additionally, when it comes to remote storage access, the time taken for RPC round trips significantly outweighs the impact of I/O size on individual I/O performance. So a promising optimization is to merge adjacent I/O requests into a single operation and fetch multiple keys with one I/O call. This approach requires a pre-prepared batch of keys for the query and the identification of I/O operations that can be combined. In this FLIP, we focus on the implementation details for batching state requests and processing them in batches.</span></p><p class="ne-p"><span class="ne-text">This FLIP firstly describes a way of batching in the </span><em><span class="ne-text">Asynchronous Execution Controller (AEC) </span></em><span class="ne-text">introduced in <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a></span><em><span class="ne-text">.</span></em><span class="ne-text"> And then for a batch of state accesses, this FLIP suggests executing smaller sub-batches in parallel while also merging neighboring I/O requests to minimize the number of I/O operations and reduce overall latency. This approach is expected to significantly boost the efficiency of state I/O, thereby enhancing the processing throughput.</span></p><h1 id="FLIP426:GroupingRemoteStateAccess-ProposedChanges">Proposed Changes</h1><p class="ne-p"><span class="ne-text">As mentioned in <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a> , there are two buffers in </span><em><span class="ne-text">Async Execution Controller</span></em><span class="ne-text"> (AEC) for state accessing requests (as shown in Fig-1), apart from keeping the order of inputs, they also help reduce the overall latency.</span></p><p class="ne-p"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image confluence-external-resource image-center" draggable="false" alt="image.png" width="390" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/17056779/1708585902010-112156e1-5efd-44e7-aa73-24078ddc3eed.png?x-oss-process=image%2Fresize%2Cw_562%2Climit_0" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/17056779/1708585902010-112156e1-5efd-44e7-aa73-24078ddc3eed.png?x-oss-process=image%2Fresize%2Cw_562%2Climit_0"></span></p><p class="ne-p" style="text-align: center;">Fig-1. Overview of Batch Execution for State</p><p class="ne-p"><span class="ne-text">There are different state tables and different request types (PUT/GET/ITERATOR/DELETE) in two buffers. The accumulation of buffer is described in <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a>, where the order of incoming requests should be preserved. When the </span><em><span class="ne-text">Active Buffer</span></em><span class="ne-text"> reaches the specified size, we group all requests in </span><em><span class="ne-text">Active Buffer</span></em><span class="ne-text"> according to the request type and state tables and then put them into execution. The following are some implementation details.</span></p><h2 id="FLIP426:GroupingRemoteStateAccess-DataStructureofBatchStateRequests"><span class="ne-text">Data Structure of Batch State Requests</span></h2><p class="ne-p"><span class="ne-text">We propose several classes to encapsulate individual user requests and define an interface that the executor should implement to support batched state execution. These classes and interfaces are internal, ensuring a clear border between the batching and execution layers. It allows for the potential integration of other StateBackends within the asynchronous State API execution framework if they implement the execution interface of batch.</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@Internal
public class StateRequest<S, K, N, UK, V, F> {
S state;
RequestType type;
K key;
N namespace;
UK userKey;
@Nullable V value;
StateFuture<F> stateFuture;
private StateRequest(S state, RequestType type, K key, @Nullable V value, InternalStateFuture<F> stateFuture) {
this.state = state;
this.type = type;
this.key = key;
this.value = value;
this.stateFuture = stateFuture;
}
public enum RequestType {
VALUE_GET,
VALUE_PUT,
LIST_GET,
LIST_PUT,
LIST_ADD,
LIST_ITER,
MAP_GET,
MAP_PUT,
MAP_ITER,
MAP_ITER_KEY,
MAP_ITER_VALUE
}
}</pre>
</div></div><p class="ne-p"><span class="ne-text">And the state executor service is defined like:</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@Experimental
public interface StateExecutor<K> {
CompletableFuture<Boolean> executeBatchRequests(Iterable<StateRequest<?, K, ?, ?, ?, ?>> stateRequests);
}</pre>
</div></div><p class="ne-p"><br/></p><p class="ne-p"><span class="ne-text">Note that this is only a draft version that may change during implementation.</span></p><h2 id="FLIP426:GroupingRemoteStateAccess-CategorizeandProcess"><span class="ne-text">Categorize and Process</span></h2><p class="ne-p"><span class="ne-text">One batch of state access requests will contain different state operation (PUT/GET/ITERATOR/DELETE) from multiple state tables. It is ensured by the </span><em><span class="ne-text">AEC</span></em><span class="ne-text"> that the requests have no order issue so they could be executed in parallel. The requests within a batch will be divided into multiple sub-groups based on different state tables and request types, so that we could merge isomorphic I/O requests into a single operation. For example, </span><span class="ne-text">as illustrated in Fig-2, </span><span class="ne-text">one batch contains multiple PUT/GET state operations from two tables, then all of state-1's GET operations will be grouped together, and aggregated into one MultiGet request which is executed by the StateExecutors to access the Remote State Storage. All PUT operations of state-1 will be grouped together, and aggregated into one WriteBatch request which is also executed by the StateExecutors.</span></p><p class="ne-p"><span class="ne-text"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image confluence-external-resource image-center" draggable="false" alt="image.png" height="400" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1706076723994-012bb70b-e62d-4ec5-a4a8-b05d53a484ca.png?x-oss-process=image%2Fresize%2Cw_1500%2Climit_0" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1706076723994-012bb70b-e62d-4ec5-a4a8-b05d53a484ca.png?x-oss-process=image%2Fresize%2Cw_1500%2Climit_0"></span></span></p><p class="ne-p" style="text-align: center;"><span class="ne-text">Fig-2. Categorize and Process State Requests </span></p><h2 id="FLIP426:GroupingRemoteStateAccess-LeverageRocksDBMultiGet"><span class="ne-text">Leverage RocksDB MultiGet</span></h2><p class="ne-p"><span class="ne-text">RocksDB supports <a class="external-link" href="https://rocksdb.org/blog/2022/10/07/asynchronous-io-in-rocksdb.html" rel="nofollow">MultiGet</a> which could improve the performance by over 100% according to different data pipelines. It could mitigate the impact of IO latency by:</span></p><ol class="ne-ol"><li><span class="ne-text">reducing the read requests: It could only read the necessary data blocks which is determined by the set of files that a given batch of keys overlaps.</span></li><li><span class="ne-text">reading asynchronously and in parallel as much as possible: It could read SST asynchronously and in parallel both in single level and multi level.</span></li></ol><p class="ne-p"><span class="ne-text">In execution of grouped isomorphic requests, it is intuitive to use the RocksDB MultiGet. However there are still some challenges using this feature:</span></p><ol class="ne-ol"><li><span class="ne-text">Currently, it is available only for PosixFileSystem. But we could still support FlinkFileSystem in <a href="https://cwiki.apache.org/confluence/x/T4p3EQ" rel="nofollow">FLIP-427</a> by proxying all async interfaces e.g. MultiRead, ReadAsync, poll, abort to corresponding methods of ForStFlieSystem and communicating with callback result handle, then ForStFlieSystem could read in parallel for different requests and notify RocksDB status.</span></li><li><span class="ne-text">Build Jar with Async IO support. We need to build RocksDB jar with folly which MultiGet uses. This will introduce more complex compile procedure and code paths.</span></li></ol><p class="ne-p"><span class="ne-text">Based on the data structure of batch state, we could leverage RocksDB MultiGet to handle the batch requests to improve the performance greatly in some scenarios even if there are some challenges that we need to overcome, we'd like also to support this in the future.</span></p><h2 class="ne-p" id="FLIP426:GroupingRemoteStateAccess-CodeExampleonHowtoAccessStateUsingBatch"><span class="ne-text">Code Example on How to<span style="color: rgb(34,34,51);"> Access State Using Batch</span></span></h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">
private <N, V> CompletableFuture<Void> processMultipleValueGetRequests(
ForStValueState<K, N, V> valueState, List<StateRequest<?, K, ?, ?>> batchValueGetRequests) {
// Get the key List from the batch state requests.
List<K> requestKeys = batchValueGetRequests.stream().map(StateRequest::getKey).collect(Collectors.toList());
CompletableFuture<Void> future = CompletableFuture.runAsync(() -> {
// Utilize multiGet to group state access IO
List<V> values = valueState.multiGet(requestKeys);
for (int i = 0; i < values.size(); i++) {
V value = values.get(i);
// Complete the stateFuture for each state request.
batchValueGetRequests.get(i).getFuture().complete(value);
}
}, stateIOExecutors);
return future;
}</pre>
</div></div><p><br/></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@Internal
public class ForStValueState {
@Override
public List<V> multiGet(List<K> keys) throws IOException {
List<byte[]> keys = serializeKeysWithGroupAndNamespace(keys);
List<byte[]> values = backend.db.multiGetAsList(keys);
List<V> result = new ArrayList<>(values.size());
for (byte[] value : values) {
result.add(valueSerializer.deserialize(value));
}
return result;
}
}</pre>
</div></div><h1 id="FLIP426:GroupingRemoteStateAccess-PublicInterfaces">Public Interfaces</h1><p><span style="color: rgb(34,34,51);">Some configurable options will be added, which allow users to adjust the behavior of batching, such as the batching size and so on.</span></p><h1 id="FLIP426:GroupingRemoteStateAccess-Compatibility,Deprecation,andMigrationPlan">Compatibility, Deprecation, and Migration Plan</h1><p class="ne-p"><span class="ne-text">A whole new code path is introduced and is fully compatible with the original one.</span></p><h1 id="FLIP426:GroupingRemoteStateAccess-TestPlan">Test Plan</h1><p class="ne-p"><span class="ne-text">New UT/ITs will be introduced for new introduced implementation above.</span></p><h1 id="FLIP426:GroupingRemoteStateAccess-RejectedAlternatives">Rejected Alternatives</h1><p class="ne-p"><span class="ne-text">None.</span></p><p class="ne-p"><br/></p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-426%3A+Grouping+Remote+State+Access">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=293046861&revisedVersion=20&originalVersion=19">View Changes Online</a>
</div>
</div>Jinzhong Li2024-02-23T02:55:45ZFLIP-424: Asynchronous State APIsZakelly Lantag:cwiki.apache.org,2009:page-293046857-332024-03-27T12:03:34Z2024-02-23T02:51:44Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~zakelly
">Zakelly Lan</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<div class="table-wrap"><table class="wrapped confluenceTable" style="letter-spacing: 0.0px;"><thead><tr><th class="confluenceTh">Discussion thread</th><td class="confluenceTd"><p><a class="external-link" href="https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0" rel="nofollow">https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0</a></p><p><a class="external-link" href="https://lists.apache.org/thread/nmd9qd0k8l94ygcfgllxms49wmtz1864" rel="nofollow">https://lists.apache.org/thread/nmd9qd0k8l94ygcfgllxms49wmtz1864</a></p></td></tr></thead><colgroup><col/><col/></colgroup><tbody><tr><th colspan="1" class="confluenceTh">Vote thread</th><td colspan="1" class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/tplyf17n3409l605bo9promf4o8tvl2j" rel="nofollow">https://lists.apache.org/thread/tplyf17n3409l605bo9promf4o8tvl2j</a></td></tr><tr><th class="confluenceTh">JIRA</th><td class="confluenceTd"><br/></td></tr><tr><th colspan="1" class="confluenceTh">Release</th><td colspan="1" class="confluenceTd">2.0</td></tr></tbody></table></div><p>Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).</p><p><style type='text/css'>/*<![CDATA[*/
div.rbtoc1711724893251 {padding: 0px;}
div.rbtoc1711724893251 ul {margin-left: 0px;}
div.rbtoc1711724893251 li {margin-left: 0px;padding-left: 0px;}
/*]]>*/</style><div class='toc-macro rbtoc1711724893251'>
<ul class='toc-indentation'>
<li><a href='#FLIP424:AsynchronousStateAPIs-Motivation'>Motivation</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-PublicInterfaces'>Public Interfaces</a>
<ul class='toc-indentation'>
<li><a href='#FLIP424:AsynchronousStateAPIs-StateFuture&FutureUtils'>StateFuture & FutureUtils</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-StateIterator'>StateIterator</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-State'>State</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-ValueState'>ValueState</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-ListState'>ListState</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-MapState'>MapState</a></li>
</ul>
</li>
<li><a href='#FLIP424:AsynchronousStateAPIs-CodeExample'>Code Example</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-InternalAPIsandimplementations'>Internal APIs and implementations</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-Compatibility,Deprecation,andMigrationPlan'>Compatibility, Deprecation, and Migration Plan</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-TestPlan'>Test Plan</a></li>
<li><a href='#FLIP424:AsynchronousStateAPIs-RejectedAlternatives'>Rejected Alternatives</a></li>
</ul>
</div></p><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p>This is a <span style="color: rgb(23,43,77);">sub-FLIP for the disaggregated state management and its related work, please read the <a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">FLIP-423</a> first to know the whole story.</span></p></div></div><h1 id="FLIP424:AsynchronousStateAPIs-Motivation"><span class="ne-text">Motivation</span></h1><p class="ne-p"><span class="ne-text">Currently, in Flink, each task processes elements sequentially within a single thread, which includes accessing the state. When employing a local disk-based state backend, accessing the state often entails I/O operations, which are markedly slower than CPU computations, making the stateful operator easily become a bottleneck of the whole job. For several years, users have frequently sought advice on performance issues related to stateful operators, particularly in jobs that manage a large state. I/O latency is the primary contributor to these issues, and it also prevents the task thread from fully exploiting CPU resources. Scaling out by increasing parallelism can mitigate the issue and enhance overall throughput, yet this approach demands additional resources, notably CPU and memory —— resources that are not the actual bottleneck.</span></p><p class="ne-p"><span class="ne-text">To maximize I/O capacity utilization and enhance the use of pre-allocated computational resources, this FLIP proposes the introduction of asynchronous state APIs. These APIs permit state access to be executed in threads separate from the task thread, returning the result when available. Consequently, the task thread can process another element while awaiting multiple pending state results. This enables concurrent processing of multiple records, ensuring that the latency of individual I/O operations no longer has a direct impact on job performance. This approach is particularly advantageous in scenarios where I/O bandwidth is underutilized and I/O latency is the limiting factor. The Disaggregated Storage Architecture, as discussed in <a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">FLIP-423</a>, is a prime example of a scenario characterized by abundant and easily scalable I/O bandwidth coupled with higher I/O latency. The asynchronous state APIs hold great promise for significantly enhancing Flink's performance when dealing with disaggregated state.</span></p><h1 id="FLIP424:AsynchronousStateAPIs-PublicInterfaces"><span class="ne-text">Public Interfaces</span></h1><p>All asynchronous state APIs employ the newly introduced Future class as their return type. We choose a new Future class instead of CompletableFuture for the following reasons:</p><ul><li><p>CompletableFuture offers robust features, including the ability for users to designate the executor for a callback. In contrast, we advocate for executing callbacks within the task thread (as outlined in <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a>) and aim to abstract these intricacies away from the user.</p></li><li><p>Prior to executing a callback, a context switch is required, a process that will be seamlessly integrated within the implementation of the new Future class.</p></li></ul><p><span style="letter-spacing: 0.0px;">The challenges for asynchronous execution will be discussed in </span><a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow" style="letter-spacing: 0.0px;">FLIP-425</a><span style="letter-spacing: 0.0px;">. While in this FLIP, we focus on the definition of the APIs.</span></p><p>A new set of APIs and interface classes are proposed, <strong>offering both synchronous API and asynchronous counterpart</strong> for state access. For async interfaces that yield a collection of elements, an iterator is provided to facilitate asynchronous loading of the elements. The new API set is named "State API V2" residing under the package path 'org.apache.flink.api.common.state.v2' and the <code>flink-core-api</code> module. They work closely with the Datastream API V2 and are annotated with @<span style="color: rgb(34,34,34);">Experimental in first few versions, and will be promoted to </span>@<span style="color: rgb(34,34,34);">PublicEvolving alongside the DataStream API V2.</span> <strong>The synchronous APIs are basically identical with the original ones</strong>, thus we only discuss the new asynchronous part only.</p><h2 id="FLIP424:AsynchronousStateAPIs-StateFuture&FutureUtils"><span class="ne-text">StateFuture & FutureUtils</span></h2><p><span class="ne-text"><code>StateFuture</code> provides selected subset of functions in <code>CompletableFuture</code> . The implementation will encapsulate a <code>CompletableFuture</code> to do the actual work. </span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>StateFuture</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@PublicEvolving
public interface StateFuture<T> {
/**
* Returns a new StateFuture that, when this future completes
* normally, is executed with this future's result as the argument
* to the supplied function.
*
* @param fn the function to use to compute the value of
* the returned StateFuture
* @param <U> the function's return type
* @return the new StateFuture
*/
<U> StateFuture<U> thenApply(Function<? super T,? extends U> fn);
/**
* Returns a new StateFuture that, when this future completes
* normally, is executed with this future's result as the argument
* to the supplied action.
*
* @param action the action to perform before completing the
* returned StateFuture
* @return the new StateFuture
*/
StateFuture<Void> thenAccept(Consumer<? super T> action);
/**
* Returns a new future that, when this future completes normally,
* is executed with this future as the argument to the supplied function.
*
* @param action the action to perform
* @return the new StateFuture
*/
<U> StateFuture<U> thenCompose(
Function<? super T, ? extends StateFuture<U>> action);
/**
* Returns a new StateFuture that, when this and the other
* given future both complete normally, is executed with the two
* results as arguments to the supplied function.
*
* @param other the other StateFuture
* @param fn the function to use to compute the value of
* the returned StateFuture
* @param <U> the type of the other StateFuture's result
* @param <V> the function's return type
* @return the new StateFuture
*/
<U,V> StateFuture<V> thenCombine
(StateFuture<? extends U> other,
BiFunction<? super T,? super U,? extends V> fn);
}</pre>
</div></div><p><br/></p><p><span>And some utilities as follows:</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>FutureUtils</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@PublicEvolving
public class FutureUtils {
/**
* Returns a completed future that does nothing and return null.
*/
public static <V> StateFuture<V> completedVoidFuture() {
// xxx
}
/**
* Creates a future that is complete once multiple other futures completed. Upon successful
* completion, the future returns the collection of the futures' results.
*
* @param futures The futures that make up the conjunction. No null entries are allowed,
* otherwise a IllegalArgumentException will be thrown.
* @return The StateFuture that completes once all given futures are complete.
*/
public static <T> StateFuture<Collection<T>> combineAll(
Collection<? extends StateFuture<? extends T>> futures) {
// xxx
}
}</pre>
</div></div><pre class="ne-codeblock language-java"><br/></pre><h2 id="FLIP424:AsynchronousStateAPIs-StateIterator"><span class="ne-text">StateIterator</span></h2><p class="ne-p"><span class="ne-text">Asynchronous iterators allow to iterate over data that comes asynchronously, on-demand.</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>StateIterator</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@PublicEvolving
public interface StateIterator<T> {
/**
* Async iterate the data and call the callback when data is ready.
* @param iterating the data action when it is ready. The return is the state future for
* chaining.
* @param <U> the type of the inner returned StateFuture's result
* @return the Future that will trigger when this iterator and all returned state future get its
* results.
*/
<U> StateFuture<Collection<U>> onNext(Function<T, StateFuture<? extends U>> iterating);
/**
* Async iterate the data and call the callback when data is ready.
* @param iterating the data action when it is ready
* @return the Future that will trigger when this iterator ends.
*/
StateFuture<Void> onNext(Consumer<T> iterating);
boolean isEmpty();
}</pre>
</div></div><h2 class="ne-codeblock language-java" id="FLIP424:AsynchronousStateAPIs-State"><code>
</code><span class="ne-text">State</span></h2><p><span class="ne-text">Base class for all state definitions:</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>State</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@PublicEvolving
public interface State {
/** Removes the value mapped under the current key. */
StateFuture<Void> asyncClear();
}</pre>
</div></div><h2 id="FLIP424:AsynchronousStateAPIs-ValueState"><span class="ne-text">ValueState</span></h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>ValueState</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@PublicEvolving
public interface ValueState<T> extends State {
/**
* Returns the current value for the state asynchronously. When the state is not partitioned the
* returned value is the same for all inputs in a given operator instance. If state partitioning
* is applied, the value returned depends on the current operator input, as the operator maintains
* an independent state for each partition. When no value was previously set using {@link #asyncUpdate(Object)},
* this will return {@code null} asynchronously.
*
* @return The {@link StateFuture} that will return the value corresponding to the current input.
*/
StateFuture<T> asyncValue();
/**
* Updates the operator state accessible by {@link #asyncValue()} to the given value. The next time
* {@link #asyncValue()} is called (for the same state partition) the returned state will represent
* the updated value. When a partitioned state is updated with {@code null}, the state for the
* current key will be removed.
*
* @param value The new value for the state.
* @return The {@link StateFuture} that will trigger the callback when update finishes.
*/
StateFuture<Void> asyncUpdate(T value);
}</pre>
</div></div><h2 class="ne-codeblock language-java" id="FLIP424:AsynchronousStateAPIs-ListState"><span class="ne-text">ListState</span></h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>ListState</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@PublicEvolving
public interface ListState<T> extends State {
/**
* Returns the current iterator for the state asynchronously. When the state is not partitioned the
* returned value is the same for all inputs in a given operator instance. If state partitioning
* is applied, the value returned depends on the current operator input, as the operator maintains
* an independent state for each partition. This method will never return (@code null).
*
* @return The {@link StateFuture} that will return list iterator corresponding to the current input.
*/
StateFuture<StateIterator<T>> asyncGet();
/**
* Updates the operator state accessible by {@link #asyncGet()} by adding the given value to the list
* of values. The next time {@link #asyncGet()} is called (for the same state partition) the returned
* state will represent the updated list.
*
* <p> Null is not allowed to be passed in
*
* @param value The new value for the state.
* @return The {@link StateFuture} that will trigger the callback when update finishes.
*/
StateFuture<Void> asyncAdd(T value);
/**
* Updates the operator state accessible by {@link #asyncGet()} by updating existing values to the
* given list of values. The next time {@link #asyncGet()} is called (for the same state partition)
* the returned state will represent the updated list.
*
* <p> Null value passed in or any null value in list is not allowed.
*
* @param values The new value list for the state.
* @return The {@link StateFuture} that will trigger the callback when update finishes.
*/
StateFuture<Void> asyncUpdate(List<T> values);
/**
* Updates the operator state accessible by {@link #asyncGet()} by adding the given values to the list
* of values. The next time {@link #asyncGet()} is called (for the same state partition) the returned
* state will represent the updated list.
*
* <p> Null value passed in or any null value in list is not allowed.
*
* @param values The new values to add for the state.
* @return The {@link StateFuture} that will trigger the callback when update finishes.
*/
StateFuture<Void> asyncAddAll(List<T> values);
}</pre>
</div></div><h2 class="ne-codeblock language-java" id="FLIP424:AsynchronousStateAPIs-MapState"><span class="ne-text">MapState</span></h2><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>MapState</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">@PublicEvolving
public interface MapState<UK, UV> extends State {
/**
* Returns the current value associated with the given key asynchronously. When the state is
* not partitioned the returned value is the same for all inputs in a given operator instance.
* If state partitioning is applied, the value returned depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @return The {@link StateFuture} that will return value corresponding to the current input.
*/
StateFuture<UV> asyncGet(UK key);
/**
* Update the current value associated with the given key asynchronously. When the state is
* not partitioned the value is updated for all inputs in a given operator instance.
* If state partitioning is applied, the updated value depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @param key The key that will be updated.
* @param value The new value for the key.
* @return The {@link StateFuture} that will trigger the callback when update finishes.
*/
StateFuture<Void> asyncPut(UK key, UV value);
/**
* Update all of the mappings from the given map into the state asynchronously. When the state is
* not partitioned the value is updated for all inputs in a given operator instance.
* If state partitioning is applied, the updated mapping depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @param map The mappings to be stored in this state.
* @return The {@link StateFuture} that will trigger the callback when update finishes.
*/
StateFuture<Void> asyncPutAll(Map<UK, UV> map);
/**
* Delete the mapping of the given key from the state asynchronously. When the state is
* not partitioned the deleted value is the same for all inputs in a given operator instance.
* If state partitioning is applied, the value deleted depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @param key The key of the mapping.
* @return The {@link StateFuture} that will trigger the callback when update finishes.
*/
StateFuture<Void> asyncRemove(UK key);
/**
* Returns whether there exists the given mapping asynchronously. When the state is
* not partitioned the returned value is the same for all inputs in a given operator instance.
* If state partitioning is applied, the value returned depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @param key The key of the mapping.
* @return The {@link StateFuture} that will return true if there exists a mapping whose key
* equals to the given key
*/
StateFuture<Boolean> asyncContains(UK key);
/**
* Returns the current iterator for all the mappings of this state asynchronously. When the state
* is not partitioned the returned iterator is the same for all inputs in a given operator instance.
* If state partitioning is applied, the iterator returned depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @return The {@link StateFuture} that will return mapping iterator corresponding to the current input.
*/
StateFuture<StateIterator<Map.Entry<UK, UV>>> asyncEntries();
/**
* Returns the current iterator for all the keys of this state asynchronously. When the state
* is not partitioned the returned iterator is the same for all inputs in a given operator instance.
* If state partitioning is applied, the iterator returned depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @return The {@link StateFuture} that will return key iterator corresponding to the current input.
*/
StateFuture<StateIterator<UK>> asyncKeys();
/**
* Returns the current iterator for all the values of this state asynchronously. When the state
* is not partitioned the returned iterator is the same for all inputs in a given operator instance.
* If state partitioning is applied, the iterator returned depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @return The {@link StateFuture} that will return value iterator corresponding to the current input.
*/
StateFuture<StateIterator<UV>> asyncValues();
/**
* Returns whether this state contains no key-value mappings asynchronously. When the state is
* not partitioned the returned value is the same for all inputs in a given operator instance.
* If state partitioning is applied, the value returned depends on the current operator input,
* as the operator maintains an independent state for each partition.
*
* @return The {@link StateFuture} that will return true if there is no key-value mapping, otherwise false.
*/
StateFuture<Boolean> asyncIsEmpty();
}</pre>
</div></div><h1 id="FLIP424:AsynchronousStateAPIs-CodeExample"><span class="ne-text">Code Example</span></h1><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p>Such usage in the following example code in<span> </span><code class="c-mrkdwn__code">processElement</code><span> </span>is an intermediate step. And as such it will be used for development and PoC purposes only. The long term solution will be discussed later.</p></div></div><p class="ne-p"><span class="ne-text">To leverage asynchronous state APIs, users should change their code accordingly:</span></p><ul class="ne-ul"><li><strong><span class="ne-text">For Table API & SQL users</span></strong><span class="ne-text">: No change.</span></li><li><strong><span class="ne-text">For DataStream API users </span></strong><span class="ne-text">or </span><strong><span class="ne-text">Table API & SQL developers</span></strong><span class="ne-text">: </span></li></ul><p class="ne-p" style="margin-left: 2.0em;"><span class="ne-text">Given a case that two streams join. Stream A is inner while Stream B is outer. Both streams have unique key. The join key contains Stream A's unique key but not B's unique key. When there is a record from Stream A coming, the code using old State APIs will be:</span></p><div class="code panel pdl" style="border-width: 1px;">
<div class="codeHeader panelHeader pdl hide-border-bottom" style="border-bottom-width: 1px;">
<b class="code-title">Synchronous state API (current)</b><span class="collapse-source expand-control" style="display:none;"><span class="expand-control-icon icon"> </span><span class="expand-control-text">Expand source</span></span><span class="collapse-spinner-wrapper"></span>
</div>
<div class="codeContent panelContent pdl hide-toolbar">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default; collapse: true" data-theme="Default">private final ValueState<RowData> aState;
private final MapState<RowData, Tuple2<RowData, Integer>> bState;
private final KeySelector<RowData, RowData> uniqueKeySelector;
public void processElement1(StreamRecord<RowData> element) throws Exception {
RowData input = element.getValue();
boolean isAccumulateMsg = RowDataUtil.isAccumulateMsg(input);
if (isAccumulateMsg) {
for (Tuple2<RowData, Integer> value : bState.values()) {
if (joinCondition.apply(input, value.f0)) {
output(input, value.f0);
RowData key = uniqueKeySelector.getKey(value.f0);
bState.put(key, Tuple2.of(value.f0, value.f1 + 1));
}
}
aState.update(input);
} else {
// omit retraction
}
}</pre>
</div>
</div><p class="ne-codeblock language-java"><span class="ne-text">And the code using new State APIs will be:</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>New APIs usage</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">private final ValueState<RowData> aState;
private final MapState<RowData, Tuple2<RowData, Integer>> bState;
private final KeySelector<RowData, RowData> uniqueKeySelector;
public void processElement1(StreamRecord<RowData> element) throws Exception {
RowData input = element.getValue();
boolean isAccumulateMsg = RowDataUtil.isAccumulateMsg(input);
if (isAccumulateMsg) {
bState.asyncValues().thenCompose((iterator) -> {
return iterator.onHasNext((value) -> {
// value is Tuple2<RowData, Integer>
if (joinCondition.apply(input, value.f0)) {
output(input, value.f0);
RowData key = uniqueKeySelector.getKey(value.f0);
return bState.asyncPut(key, Tuple2.of(value.f0, value.f1 + 1));
} else {
return FutureUtils.emptyFuture();
}
});
}).thenAccept((e) -> {
return aState.asyncUpdate(input);
});
// or just:
// aState.asyncUpdate(input);
} else {
// omit retraction
}
}</pre>
</div></div><p><br/></p><p><span class="ne-text" style="color: rgb(23,43,77);">The state APIs described above are flexible and allowing user to define the execution order as wished. By providing a callback via <code>thenApply</code> , <code>thenCompose</code> or <code>thenCombine</code> of a StateFuture, user could easily assemble their tasks using multiple code segments. </span>The interface and functionality of <code>StateFuture</code> mirror those of <code>CompletableFuture</code> , which is extensively adopted and proven to be complete and powerful enough.</p><h1 id="FLIP424:AsynchronousStateAPIs-InternalAPIsandimplementations">Internal APIs and implementations</h1><p>The original state-related internal APIs and implementation are designed based on single-thread execution model, so they may be not suitable for new execution model where multiple state requests are on going. For example, the KeyedStateBackend holds the key context and the internal states hold the namespaces. It is better not to provide these context-related methods in global instances. Thus we will add new interface classes and implementations as needed, and left the old ones untouched. This FLIP will not list them in detail as we cannot predict all situations until we finished our implementations.</p><p><br/></p><h1 id="FLIP424:AsynchronousStateAPIs-Compatibility,Deprecation,andMigrationPlan"><span class="ne-text" style="color: rgb(23,43,77);">Compatibility, Deprecation, and Migration Plan</span></h1><p><span class="ne-text">The newly introduced APIs are designed to provide synchronous and asynchronous APIs. The code path for the new APIs is completely independent from the original one. We strongly recommend that users stick to either the synchronous APIs or asynchronous APIs exclusively, rather than mixing their usage. Although using both the asynchronous and synchronous APIs simultaneously won't cause compatibility issues, it may lead to suboptimal performance. The synchronous API can block the task thread until its execution is complete, potentially resulting in a performance regression when compared to using the asynchronous state calls exclusively. But still, the mixed usage could out-perform the sync APIs usage only, a code example is:</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">bState.asyncValues().thenApply((iterator) -> {
iterator.onHasNext((value) -> {
if (joinCondition.apply(input, value.f0)) {
output(input, value.f0);
RowData key = uniqueKeySelector.getKey(value.f0);
bState.put(key, Tuple2.of(value.f0, value.f1 + 1));
}
});
});
aState.update(input);</pre>
</div></div><p>While the <code>update</code> or <code>put</code> is sync state access, the state iteration is executed asynchronously. The update and put are relatively lightweight methods compared with the iteration, so they won't block too much the asynchronous pipeline and the job could perform better than pure synchronous execution. Compared with the example of stream join using pure asynchronous APIs, the mixed usage like this cannot totally compete with the asynchronous one in performance, but it is much more easier to use, especially for those users who <span style="color: rgb(34,34,34);">are</span> not familiar with writing asynchronous programs and want to migrate their job from Flink 1.x to 2.0. We plan to allow the mixed usage considering user-friendliness, but warn them about the performance concern in runtime.</p><p><strong>The original APIs, will be deprecated alongside the DataStream V1 APIs after Flink 2.0, we will discuss this in future.</strong></p><h1 id="FLIP424:AsynchronousStateAPIs-TestPlan"><span class="ne-text" style="color: rgb(23,43,77);">Test Plan</span></h1><p class="ne-p"><span class="ne-text">New UT/ITs will be introduced for asynchronous APIs. New E2E tests of jobs using asynchronous state APIs will also be delivered.</span></p><h1 id="FLIP424:AsynchronousStateAPIs-RejectedAlternatives"><span class="ne-text">Rejected Alternatives</span></h1><ul class="ne-ul"><li><strong><span class="ne-text">Batch State APIs: </span></strong><span class="ne-text">The core idea is to batch the elements first, and user should write code processing a batch of elements and accessing the state with a batch of keys. </span></li></ul><ul class="ne-list-wrap"><ul class="ne-ul"><li><strong><span class="ne-text">Pros</span></strong><span class="ne-text">: </span></li></ul></ul><ul class="ne-list-wrap"><ul class="ne-list-wrap"><ul class="ne-ul"><li><span class="ne-text">The state execution is more simpler, no mixed updates and gets should be categorized.</span></li><li><span class="ne-text">Less intermediated memory consumption. There is only one callback and context for a batch of state access.</span></li></ul></ul></ul><ul class="ne-list-wrap"><ul class="ne-ul"><li><strong><span class="ne-text">Cons</span></strong><span class="ne-text">:</span></li></ul></ul><ul class="ne-list-wrap"><ul class="ne-list-wrap"><ul class="ne-ul"><li><span class="ne-text">Complicated user code with batch processing. </span></li><li><span class="ne-text">Expose Key Context to user.</span></li><li><span class="ne-text">The state iterator for a batch of keys is hard to understand and difficult to use.</span></li></ul></ul></ul>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-424%3A+Asynchronous+State+APIs">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=293046857&revisedVersion=33&originalVersion=32">View Changes Online</a>
</div>
</div>Zakelly Lan2024-02-23T02:51:44ZFLIP-425: Asynchronous Execution ModelYanfei Leitag:cwiki.apache.org,2009:page-293046859-502024-03-27T12:03:16Z2024-02-23T02:53:30Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~fredialei
">Yanfei Lei</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<div class="table-wrap"><table class="wrapped confluenceTable" style="letter-spacing: 0.0px;"><thead><tr><th class="confluenceTh">Discussion thread</th><td class="confluenceTd"><p><a class="external-link" href="https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0" rel="nofollow">https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0</a></p><p><a class="external-link" href="https://lists.apache.org/thread/wxn1j848fnfkqsnrs947wh1wmj8n8z0h" rel="nofollow">https://lists.apache.org/thread/wxn1j848fnfkqsnrs947wh1wmj8n8z0h</a></p></td></tr></thead><colgroup><col/><col/></colgroup><tbody><tr><th colspan="1" class="confluenceTh">Vote thread</th><td colspan="1" class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/drglfqv83z5n4hybdjn0lvcc3xjkrtk0" rel="nofollow">https://lists.apache.org/thread/drglfqv83z5n4hybdjn0lvcc3xjkrtk0</a></td></tr><tr><th class="confluenceTh">JIRA</th><td class="confluenceTd"><br/></td></tr><tr><th colspan="1" class="confluenceTh">Release</th><td colspan="1" class="confluenceTd">2.0</td></tr></tbody></table></div><p>Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).</p><p><style type='text/css'>/*<![CDATA[*/
div.rbtoc1711724893283 {padding: 0px;}
div.rbtoc1711724893283 ul {margin-left: 0px;}
div.rbtoc1711724893283 li {margin-left: 0px;padding-left: 0px;}
/*]]>*/</style><div class='toc-macro rbtoc1711724893283'>
<ul class='toc-indentation'>
<li><a href='#FLIP425:AsynchronousExecutionModel-Motivation'>Motivation</a>
<ul class='toc-indentation'>
<li><a href='#FLIP425:AsynchronousExecutionModel-Contextmaintenanceandswitch'>Context maintenance and switch</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Necessaryorderpreservation'>Necessary order preservation</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Watermarksandtimers'>Watermarks and timers</a></li>
</ul>
</li>
<li><a href='#FLIP425:AsynchronousExecutionModel-ProposedChanges'>Proposed Changes</a>
<ul class='toc-indentation'>
<li><a href='#FLIP425:AsynchronousExecutionModel-AsyncExecutionController'>Async Execution Controller</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Callbackexecution'>Callback execution</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-ElementOrder'>Element Order</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-CheckpointTrigger'>Checkpoint Trigger</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Timers'>Timers</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Watermark'>Watermark</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-OtherNon-recordEvents'>Other Non-record Events</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-RecordContext'>Record Context</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-FasterCheckpointDrain'>Faster Checkpoint Drain</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-ErrorHandling'>Error Handling</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Configurations'>Configurations</a></li>
</ul>
</li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Performanceanalysis'>Performance analysis</a>
<ul class='toc-indentation'>
<li><a href='#FLIP425:AsynchronousExecutionModel-PoCTest'>PoC Test</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Per-recordoverhead'>Per-record overhead</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Strictly-orderedmodevs.Out-of-ordermodeforwatermarkprocessing'>Strictly-ordered mode vs. Out-of-order mode for watermark processing</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-SynchronousexecutionwithasynchronousAPIs'>Synchronous execution with asynchronous APIs</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-ComparedwithBatch-styleAPIs'>Compared with Batch-style APIs</a></li>
</ul>
</li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Compatibility,Deprecation,andMigrationPlan'>Compatibility, Deprecation, and Migration Plan</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-TestPlan'>Test Plan</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-RejectedAlternatives'>Rejected Alternatives</a></li>
<li><a href='#FLIP425:AsynchronousExecutionModel-Appendix:'>Appendix:</a>
<ul class='toc-indentation'>
<li><a href='#FLIP425:AsynchronousExecutionModel-HowtorunSynchronousexecutionwithasynchronousAPIs'>How to run Synchronous execution with asynchronous APIs</a></li>
</ul>
</li>
</ul>
</div></p><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p><span style="color: rgb(51,51,51);">This is a<span> </span></span><span style="color: rgb(23,43,77);">sub-FLIP for the disaggregated state management and its related work, please read the<span> </span><a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">FLIP-423</a><span> </span>first to know the whole story.</span></p></div></div><p><br/></p><h1 id="FLIP425:AsynchronousExecutionModel-Motivation">Motivation</h1><p><span class="ne-text"><a href="https://cwiki.apache.org/confluence/x/SYp3EQ" rel="nofollow">FLIP-424</a> introduces asynchronous state APIs with callbacks allowing state access to be executed in threads separate from the task thread, making better usage of I/O bandwidth and enhancing throughput. This FLIP proposes an execution framework for asynchronous state APIs. The execution code path for the new API is completely independent from the original one, where many runtime components are redesigned. We intend to delve into the challenges associated with asynchronous execution and provide an in-depth design analysis for each module. Furthermore, we will conduct a performance analysis of the new framework relative to the current implementation and examine how it measures up against other potential alternatives.</span></p><p class="ne-p"><span class="ne-text">Asynchronous state APIs can improve system responsiveness and concurrency capabilities, but the out-of-order execution also brings in a series of challenges:</span></p><h4 class="ne-p" id="FLIP425:AsynchronousExecutionModel-Contextmaintenanceandswitch"><strong><span class="ne-text">Context maintenance and switch</span></strong></h4><p class="ne-p"><span class="ne-text">Each input element may involve several state accesses. In asynchronous state execution, the whole process will be split into several stages defined by chained futures and callbacks. They will be executed in task thread in order but code segments for different inputs will be mixed together. There should be a </span><em><span class="ne-text">context</span></em><span class="ne-text"> for each callback execution recording any necessary runtime information. This also helps track whether an input element finished processing eventually.</span></p><h4 class="ne-p" id="FLIP425:AsynchronousExecutionModel-Necessaryorderpreservation"><strong><span class="ne-text">Necessary order preservation</span></strong></h4><p class="ne-p"><span class="ne-text">A record may contain multiple state requests, and state requests for different records may be interspersed with each other under the asynchronous state APIs, which may introduce some concurrency issues, such as 'dirty read'. Give the case below:</span></p><p class="ne-p" style="text-align: center;"><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource" draggable="false" height="358" width="592" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706167134681-d6049063-e6bb-405f-8c13-8221ce600788.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706167134681-d6049063-e6bb-405f-8c13-8221ce600788.png"></span></p><p class="ne-p" style="text-align: center;"><span style="color: rgb(138,143,141);">Fig 1. "Dirty read"</span></p><p class="ne-p" style="text-align: center;"><br/></p><p class="ne-p"><span class="ne-text">The records with the same key manipulate the same state entry, thus the execution order of their updates and reads will affect the final state. The new runtime framework should ensure that the execution order of those same-key records is effectively equivalent to the synchronous execution. </span></p><h4 class="ne-p" id="FLIP425:AsynchronousExecutionModel-Watermarksandtimers"><strong><span class="ne-text">Watermarks and timers</span></strong></h4><p class="ne-p"><span class="ne-text">Since the inputs are processed and finished out-of-order, the watermark sent to downstream and the timer trigger should be reconsidered with more cautious.</span></p><h1 id="FLIP425:AsynchronousExecutionModel-ProposedChanges">Proposed Changes</h1><p class="ne-p"><span class="ne-text">With all the above challenges, this FLIP proposes an asynchronous state execution framework. The whole picture of the framework is as follows:</span></p><p class="ne-p"><br/></p><p class="ne-p" style="text-align: center;"><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image" draggable="false" width="700" src="https://cwiki.apache.org/confluence/download/attachments/293046859/image-2024-3-5_14-24-31.png?version=1&modificationDate=1709619872000&api=v2" data-image-src="https://cwiki.apache.org/confluence/download/attachments/293046859/image-2024-3-5_14-24-31.png?version=1&modificationDate=1709619872000&api=v2" data-unresolved-comment-count="0" data-linked-resource-id="294816294" data-linked-resource-version="1" data-linked-resource-type="attachment" data-linked-resource-default-alias="image-2024-3-5_14-24-31.png" data-base-url="https://cwiki.apache.org/confluence" data-linked-resource-content-type="image/png" data-linked-resource-container-id="293046859" data-linked-resource-container-version="50" alt=""></span></p><p class="ne-p" style="text-align: center;"><span style="color: rgb(138,143,141);">Fig 2. A whole picture under asynchronous state API</span></p><p class="ne-p"><span class="ne-text">The user's original code in processElement will be divided into several stages by futures and callbacks, with processElement serving as the entry point for input processing and individual callbacks dedicated to each state access. Consequently, the execution will become significantly more complex than before. Our design is founded on the following prerequisites:</span></p><ol class="ne-ol"><li><span class="ne-text">All user code segments (processElement, callbacks, onTimer etc.) will be executed in </span><strong><span class="ne-text">a single task thread</span></strong><span class="ne-text"> as before.</span></li><li><span class="ne-text">The order of elements with identical partitioned keys will be preserved, while allowing out-of-order execution for elements with different keys.</span></li><li><span class="ne-text">Completed state access results, along with their callbacks, will be encapsulated in a mail that is inserted into the Mailbox. These mails will be processed with a higher priority than normal input processing.</span></li><li><span class="ne-text">Control the total amount of data being processed and pause the processing of input if necessary.</span></li></ol><p class="ne-p"><span class="ne-text">The following chapters will introduce each part in detail.</span></p><h2 id="FLIP425:AsynchronousExecutionModel-AsyncExecutionController"><span class="ne-text">Async Execution Controller</span></h2><p><span class="ne-text"><br/><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource image-center" draggable="false" height="400" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706796606462-456a9a06-e987-4a69-b668-77377bab7659.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706796606462-456a9a06-e987-4a69-b668-77377bab7659.png"></span></span></p><p class="ne-p" style="text-align: center;"><br/></p><p class="ne-p" style="text-align: center;"><span style="color: rgb(138,143,141);">Fig 3. Async Execution Controller</span></p><p class="ne-p" style="text-align: left;"><span class="ne-text">As the most complicated component, the </span><strong><em><span class="ne-text">Async Execution Controller (AEC)</span></em></strong><strong><span class="ne-text"> </span></strong><span class="ne-text">receives state requests from user, and put them into execution according to some strategies. It is responsible for:</span></p><ul class="ne-ul"><li><span class="ne-text">Strictly preserving the sequence of elements bearing the same key by delaying subsequent requests until the processing of preceding ones is finalized. This is achieved by the </span><strong><em><span class="ne-text">Key Accounting unit</span></em></strong><span class="ne-text">, which holds the current in-flight key and tracks the corresponding ongoing records. For more details, please refer to 'Element Order'. The incoming state requests will split by this unit and push into different buffers:</span></li></ul><ul class="ne-list-wrap"><ul class="ne-ul"><li><strong><span class="ne-text">Active buffer</span></strong><span class="ne-text">: the requests in this buffer could be executed when the buffer is full or timeout is reached.</span></li><li><strong><span class="ne-text">Blocking buffer</span></strong><span class="ne-text">: the requests in that should wait until all preceding records with identical key finishing its execution. After which the queueing requests will move into the active buffer.</span></li></ul></ul><ul class="ne-ul"><li><span class="ne-text">Tracking the in-flight data(records) and blocking the input if too much data in flight (back-pressure). It invokes yield() to pause current operations, allowing for the execution of callbacks (mails in Mailbox).</span></li></ul><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p>Please note that the AEC utilizes 'records' (StreamRecord) to regulate the volume of in-flight data. Consequently, when a state request's callback triggers additional state requests, all these requests are associated with a single record that has already been authorized by the AEC to proceed. Therefore, they will not be impeded, and there will be no potential risk of stack overflow for recursive invocation of yield().</p></div></div><ul class="ne-ul"><li><span class="ne-text">To make full use of memory, the above two buffers also have the function of batching state accessing requests, which will be introduced in detail in <a href="https://cwiki.apache.org/confluence/x/TYp3EQ" rel="nofollow">FLIP-426 Access Remote State in Batch</a>.</span></li></ul><p class="auto-cursor-target"><strong>Pseudo-code</strong> when there is an incoming StateRequest that arrives AEC:</p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">// incoming state request
if (!request.getRecord().canProceed()) {
// apply for processing
while (inFlightRecords >= limit) {
// too many inflight records, yield for callback execution.
yield();
}
inFlightRecords++;
permitToProceed(request.getRecord());
}
if (keyAccountingUnit.availableKey(request.getKey())) {
keyAccountingUnit.occupyKey(request.getKey);
put(activeBuffer, request);
} else {
put(blockingBuffer, request);
}
// run if needed</pre>
</div></div><p><br/></p><h2 id="FLIP425:AsynchronousExecutionModel-Callbackexecution"><span class="ne-text">Callback execution</span></h2><p class="ne-p"><span class="ne-text">The </span><code class="ne-code"><span class="ne-text">StateFuture</span></code> allows for the registration of a callback through its <code>then()</code> method, which will be invoked once the result of the state access is ready. We proposed executing these callbacks within the task thread, the primary benefits include:</p><ol class="ne-ol"><li><span class="ne-text">Prevention of concurrency issues when interacting with internal variables.</span></li><li><span class="ne-text">Streamlined management of the execution order for callbacks and input processing.</span></li><li><span class="ne-text">More precise semantics for asynchronous state access, concentrating solely on parallel state retrieval without involving other CPU-intensive parts of the user code.</span></li></ol><p class="ne-p"><span class="ne-text">The framework will execute a hook after the state request is finished. This hook encapsulates the callback function offered by </span><code class="ne-code"><span class="ne-text">then()</span></code><span class="ne-text"> method as a mail and delivers it to the mailbox executor, allowing the task thread to execute the callback function.</span></p><p class="ne-p"><span class="ne-text">According to the existing implementation of the mailbox, the priority of the callback is higher than records. If there are both new records and old records' callbacks existing in the mailbox at the same time, callbacks will be processed first.</span></p><p class="ne-p"><br/></p><p class="ne-p"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource image-center" draggable="false" alt="Fig 4. Callback execution" width="707" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706164626295-e6f17425-ac63-442c-a5ca-a90456bfe3d2.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706164626295-e6f17425-ac63-442c-a5ca-a90456bfe3d2.png"></span></p><p class="ne-p" style="text-align: center;"><span style="color: rgb(138,143,141);">Fig 4. Callback execution</span></p><p class="ne-p"><br/></p><p class="ne-p"><span class="ne-text">Here is a possible optimization: when the state request can return quickly, the callback can be executed immediately instead of being delivered to the mailbox, which is equivalent to synchronous execution.</span></p><h2 id="FLIP425:AsynchronousExecutionModel-ElementOrder"><span class="ne-text">Element Order</span></h2><p class="ne-p"><span class="ne-text">As described above, the user code will be executed out-of-order, as well as different records. To strictly ensure the consistency of results from synchronous and asynchronous running modes, it is proposed that:</span></p><ul class="ne-ul"><li><strong><span class="ne-text">For records with the same key</span></strong><span class="ne-text">, only one is allowed to be processed at the same time. The following record will be buffered and blocked until the preceding same-key record finishes all its chained callbacks.</span></li><li><strong><span class="ne-text">For records with different keys</span></strong><span class="ne-text">, out-of-order execution is allowed.</span></li></ul><p class="ne-p"><span class="ne-text">The </span><strong><span class="ne-text">Key Accounting Unit</span></strong><span class="ne-text"> is used to preserve the ordering between two independent chained </span><code class="ne-code"><span class="ne-text">StateFuture </span></code><span class="ne-text">statements. For the "Dirty read" case in Fig.1:</span></p><ul class="ne-ul"><li><span class="ne-text">For the synchronous interface, everything is executed in sequence, and no extra work needs to be done.</span></li><li><span class="ne-text">For the asynchronous interface, Record A should run with </span><em><span class="ne-text">Read</span></em><span class="ne-text">, </span><em><span class="ne-text">Update</span></em><span class="ne-text"> and </span><em><span class="ne-text">Output</span></em><span class="ne-text">, while Record B should stay at </span><strong><span class="ne-text">Blocking buffer.</span></strong></li></ul><p class="ne-p"><span class="ne-text">The core data structure within this Unit is to remember the ongoing record and its keys. A Map<Key, Record> is suitable in this scenario. The pseudo-code of classification state access is:</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">final Map<Key, Record> accounting;
public void buffer(Record record, Key key, StateAccess access) {
Record previous = accounting.putIfAbscent(key, record);
if (previous == null || previous == record) {
collectToActive(key, access);
} else {
collectToBlocking(key, access);
}
}
</pre>
</div></div><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">T</span><span class="ne-text">he challenge arises in determining when all the processing logic associated with Record A is fully executed. To address this, we have adopted a reference counting mechanism that tracks ongoing operations (either processing input or executing a callback) related to a single record. A record is considered completely processed when there are no remaining references from these operations. To ensure clarity and proper tracking, we have made four rules:</span></p><ol class="ne-ol"><li><span class="ne-text">For all keyed stream operators, the count increases by 1 before </span><code class="ne-code"><span class="ne-text">Input#processElement</span></code><span class="ne-text">, and the count decreases by 1 after </span><code class="ne-code"><span class="ne-text">Input#processElement</span></code><span class="ne-text">.</span></li><li><span class="ne-text">When calling </span><code class="ne-code"><span class="ne-text">then()</span></code><span class="ne-text">, the count increases by 1.</span></li><li><span class="ne-text">After the callback code block in </span><code class="ne-code"><span class="ne-text">then()</span></code><span class="ne-text"> is executed, the count decreases by 1.</span></li><li><span class="ne-text">When the count is reduced to 0, it is considered that all operations on a record have finished.</span></li></ol><p><span class="ne-text"><br/><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource image-center" draggable="false" alt="Fig 5. The execution timeline of a record" width="734" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706170198019-1f3b1915-b58f-405c-bda6-f2860fd5c5a8.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706170198019-1f3b1915-b58f-405c-bda6-f2860fd5c5a8.png"></span></span></p><p style="text-align: center;"><span class="ne-text"><span style="color: rgb(138,143,141);">Fig 5. The execution timeline of a record</span></span></p><p class="ne-p"><span class="ne-text">The specific counting is as shown above when Record A is processing. ①②, ③④ and ⑤⑥ are corresponding pairs of increments and decrements that aim to track pending single code block. When the count reaches 0 after ⑥, the operation of B in the </span><strong><span class="ne-text">Blocking buffer</span></strong><span class="ne-text"> will be moved to the </span><strong><span class="ne-text">Active buffer</span></strong><span class="ne-text">. </span></p><p class="ne-p"><span class="ne-text">To cover the statements without a callback, in addition to the reference count marked in Fig.5, each state request itself is also protected by a paired reference count.</span></p><p class="ne-p"><span class="ne-text">The counting update is totally managed by the framework and transparent to users.</span></p><p class="ne-p"><br/></p><p class="ne-p"><strong><span class="ne-text">Strict order of 'processElement'</span></strong></p><p>According to the design above, the <em>AEC</em> takes control of the execution of state requests and corresponding records. However, if a StreamRecord in a keyed stream does not touch state, <em>AEC</em> loses control of it. It disrupts the order of records with the same key, while in some cases resulting in correctness issues. To address this, we introduce a more strict mode keeping the order of invoking 'processElement':</p><ul><li><strong>Record-ordered (default)</strong>: The records with same keys are strictly processed in order of arrival.</li><li><strong>State-ordered</strong>: For same-key records, state requests and subsequent callbacks are processed in the order in which each record makes its first state request. But the code before the first state request for each record can be processed out-of-order with requests from other records.</li></ul><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p>The <strong>State-ordered </strong>is only for internal usage. We won't expose this option to users in first version.</p></div></div><p>We have introduced the <em>state-ordered</em> mode above, and for a more strict mode <em>record-ordered,</em> there will be some change before invoking 'processElement'. To maximumly reuse the logic of <em>AEC</em>, in <em>record-ordered</em> mode, an 'empty' state request is sent to <em>AEC</em> with 'processElement' as its callback. It does nothing but invoke the callback once the <em>AEC</em> thinks it can be processed (be put into the <em>Active Buffer</em>). Thus the 'processElement' encapsulated into the 'empty' state request will follow the logic of <em>AEC</em> above and start processing after all preceding same-key records finish their execution, otherwise it will stay in <em>Blocking Buffer</em> or call <em>yield()</em> if there are too many in-flight records. </p><h2 id="FLIP425:AsynchronousExecutionModel-CheckpointTrigger"><span class="ne-text" style="color: rgb(0,0,0);">Checkpoint Trigger</span></h2><p><span class="ne-text" style="color: rgb(0,0,0);">As described in the "Async Execution Controller(</span><em><span class="ne-text" style="color: rgb(0,0,0);">AEC</span></em><span class="ne-text" style="color: rgb(0,0,0);">)" section, the in-flight state requests will be tracked in the </span><em><span class="ne-text" style="color: rgb(0,0,0);">Acitve Buffer</span></em><span class="ne-text" style="color: rgb(0,0,0);"> and </span><em><span class="ne-text" style="color: rgb(0,0,0);">Blocking Buffer</span></em><span class="ne-text" style="color: rgb(0,0,0);">. When the Task starts to perform a checkpoint, all in-flight data must be drained before triggering the checkpoint of StateBackend. The incoming data in the AEC will be blocked until the "drain" operation is complete. </span></p><p><span style="color: rgb(0,0,0);"><span class="ne-text">In more detail, </span><em><span class="ne-text">AEC</span></em> uses a <code>inFilghtReocordNum</code> variable to trace the current number <span class="ne-text">of records in progress. Every time the AEC receives a new record, the <code>inFilghtReocordNum</code> increases by 1; when all processing and callback for this record have completed, the <code>inFilghtReocordNum</code> decreases by 1. When processing one checkpoint mail, the current task thread will give up the time slice through the </span><code class="ne-code"><span class="ne-text">yield()</span></code><span class="ne-text"> method of the mailbox executor, so that the ongoing state request’s callback and the blocking state requests will be drained first until </span><code class="ne-code"><span class="ne-text">inFlightRecordNum</span></code><span class="ne-text"> reduces to 0.</span></span></p><p class="ne-p"><strong><span class="ne-text" style="color: rgb(0,0,0);">1) Impact of draining on job performance</span></strong></p><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">The checkpoint draining operation is not expected to have a significant impact on job performance, </span><span class="ne-text">as it processes ongoing user data much like it does during runtime. </span><span class="ne-text" style="color: rgb(51,51,51);">In other words, from the user's perspective, there is almost no interruption in data processing and output during checkpoint </span><span class="ne-text" style="color: rgb(0,0,0);">draining</span><span class="ne-text" style="color: rgb(51,51,51);">. </span></p><p class="ne-p"><strong><span class="ne-text" style="color: rgb(0,0,0);">2) Impact of draining on checkpoint duration</span></strong></p><p class="ne-p"><span class="ne-text">The AEC controls the maximum number of in-flight records, which means that the number of records that need to be drained during the checkpoint synchronous phase is controllable. When the collision rate of blocking keys among these in-flight records is low, the in-flight data can be expeditiously processed in parallel. </span><span class="ne-text" style="color: rgb(51,51,51);">Conversely, a high collision rate necessitates that the conflicting keys be processed sequentially. Fortunately, the</span><span class="ne-text"> cache in a block or file could help ensure the state same-key access performance. Overall, although the draining operation may prolong the checkpoint synchronous duration, its impact is controllable.</span></p><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">How to build checkpoints for disaggregated state storage is beyond the scope of this FLIP, and <a href="https://cwiki.apache.org/confluence/x/UYp3EQ" rel="nofollow">FLIP-428</a> will discuss how to integrate checkpoint and optimize the checkpoint end-to-end duration by accelerating the checkpoint asynchronous phase.</span></p><h2 id="FLIP425:AsynchronousExecutionModel-Timers"><span class="ne-text">Timers</span></h2><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">Timers are used to schedule actions for later (event-time or processing-time), such as firing a window, or calling back a ProcessFunction. </span></p><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">In Flink, the timer is a special state, and the scheduling of timers will access and update the timer state. And the callback of timers (such as </span><code class="ne-code"><span class="ne-text" style="color: rgb(0,0,0);">KeyedProcessFunction#onTimer()</span></code><span class="ne-text" style="color: rgb(0,0,0);">) may access other keyed states. Let's discuss these two parts separately.</span></p><p><strong><span class="ne-text">1. Timer as a special state</span></strong><span class="ne-text">. </span></p><p class="ne-p"><span class="ne-text">Currently, Flink provides two implementations for storing timers, </span><a class="external-link" href="https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/ops/state/state_backends/#timers-heap-vs-rocksdb" rel="nofollow"><span class="ne-text">timers can be stored on the JVM heap or RocksDB</span></a><span class="ne-text">. Asynchronous access to timer may complicate timer scheduling, for convenience, in basic version of the framework, we plan to directly use the JVM heap to store timers, and the registration and access of timers are synchronous. </span></p><p><strong><span class="ne-text">2. Other keyed state's operations in the callback of timers.</span></strong><span class="ne-text"> </span></p><p class="ne-p"><span class="ne-text">It should be emphasized that </span><a class="external-link" href="https://www.ververica.com/blog/4-characteristics-of-timers-in-apache-flink" rel="nofollow"><span class="ne-text">timers are registered on a KeyedStream</span></a><span class="ne-text">. The callback of a timer is also encapsulated as a mail, which would be </span><a class="external-link" href="https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/dev/datastream/operators/process_function/#timers" rel="nofollow"><span class="ne-text">executed synchronously</span></a><span class="ne-text"> in the task thread. Hence, the trigger of timers can be regarded as a special record, and the </span><strong><span class="ne-text">Key Accounting Unit </span></strong><span class="ne-text">described in the section "Element Order" also can be used to ensure the order of state operation of timers and other records. The subtle difference is that the reference count of one record also should increase/decrease by 1 around </span><code class="ne-code"><span class="ne-text">WindowOperator#onEventTime()/onProcessingTime()</span></code><span class="ne-text">and </span><code class="ne-code"><span class="ne-text">KeyedProcessOperator#onEventTime()/onProcessingTime()</span></code><span class="ne-text">.</span></p><h2 id="FLIP425:AsynchronousExecutionModel-Watermark"><span class="ne-text">Watermark</span></h2><p class="ne-p"><span class="ne-text">The mechanism in Flink to measure progress in event time is watermark. As the watermarks flow through the streaming program, they advance the event time at the operators where they arrive. Whenever an operator advances its event time, it generates a new watermark downstream for its successor operators. </span></p><p class="ne-p"><span class="ne-text">There is an implicit order requirement between watermarks and records, the </span><strong><span class="ne-text">watermark</span></strong><span class="ne-text"> should be emitted to downstream when all preceding records are finished. </span><span class="ne-text">To achieve these under asynchronous state APIs, we propose a mechanism that segments inputs into distinct </span><em><span class="ne-text">epochs</span></em><span class="ne-text">, marked by the arrival of watermark. Records are assigned to a unique </span><em><span class="ne-text">epoch</span></em><span class="ne-text"> based on their arrival. If all the records belonging to one </span><em><span class="ne-text">epoch</span></em><span class="ne-text"> finished their execution, the watermark right behind those records should be processed (</span><span class="ne-text">advance event-time && </span><span class="ne-text">sent to downstream) immediately. There are three statuses for an epoch:</span></p><ul class="ne-ul"><li><strong><span class="ne-text">Open</span></strong><span class="ne-text">: The latter non-record input has not arrived. So arriving records will be collected into this epoch.</span></li><li><strong><span class="ne-text">Closed: </span></strong><span class="ne-text">The records belonging to this epoch are settled since the following non-record input has arrived.</span></li><li><strong><span class="ne-text">Finished: </span></strong><span class="ne-text">The records of this epoch have finished execution after the epoch is closed.</span></li></ul><p class="ne-p"><span class="ne-text">A module named </span><em><span class="ne-text">EpochManager</span></em><span class="ne-text"> is invented to handle the epoch using a </span><strong><span class="ne-text">queue</span></strong><span class="ne-text">, making sure a watermark behind an epoch should be triggered after all records in or before this epoch have finished processing. Besides this, </span><em><span class="ne-text">EpochManager</span></em><span class="ne-text"> offers two modes for the execution order of a watermark and <span class="ne-text">subsequent</span> records:</span></p><ul><li class="ne-p"><strong><span class="ne-text">Strictly-ordered:</span></strong><span class="ne-text"> Latter records should wait until the preceding watermark is triggered.</span></li><li class="ne-p"><strong><span class="ne-text">Out-of-order (default):</span></strong><span class="ne-text"> Latter records and </span><span class="ne-text">preceding watermark can be processed out-of-order. </span></li></ul><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p><span style="color: rgb(34,34,34);">The semantics of watermarks do not define the sequence between a watermark and subsequent records. For the most part, this is inconsequential, except it may affect some current users who have previously relied on the implicit assumption of an ordered execution.<span> In the first version, we initially support only out-of-order processing. We may consider exposing the 'Strictly-ordered' mode once we encounter a concrete use case that necessitates it.</span></span></p></div></div><p class="ne-p"><span class="ne-text">Here we introduce the<strong> o</strong><em><strong>ut-of-order</strong></em></span><span class="ne-text"> execution mode:</span></p><ul class="ne-ul"><li><span class="ne-text">There is only one epoch open. Collect arriving records for that epoch.</span></li><li><span class="ne-text">If a non-record input (watermark) arrives, close the current open epoch and register a callback for it. Open another epoch collecting the records.</span></li><li><span class="ne-text">Count the pending records for each non-finished epoch, finish an epoch if all records finished execution.</span></li><li><span class="ne-text">Trigger the callback when:</span></li></ul><ul class="ne-list-wrap"><ul class="ne-ul"><li><span class="ne-text">An epoch's status transit to Finished. </span></li><li><strong><span class="ne-text">AND</span></strong><span class="ne-text"> it is the very first epoch in the queue.</span></li></ul></ul><ul class="ne-ul"><li><span class="ne-text">Remove the epoch after triggering.</span></li></ul><p class="ne-p" style="text-align: center;"><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource" draggable="false" alt="Fig 6. The order between watermarks and records" width="502" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706247516773-6f7b8e12-f90f-4e76-9a07-68743401e4e7.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706247516773-6f7b8e12-f90f-4e76-9a07-68743401e4e7.png"></span></p><p class="ne-p" style="text-align: center;"><span style="color: rgb(138,143,141);">Fig 6. The order between watermarks and records in out-of-order mode</span></p><p class="ne-p" style="text-align: left;"><span style="color: rgb(0,0,0);"><span class="ne-text">In o<em>ut-of-order</em> mode, records in subsequent epochs may be processed before the timer triggering brought by watermark, which is different from the behavior under the synchronous state APIs. In strictly-ordered mode, all subsequent inputs will be blocked until the current epoch is finished, which may bring in a small drop in throughput</span><span class="ne-text">. User may choose between the two modes according to their needs, to achieve a better performance or strictly state access order semantics.</span></span></p><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">For jobs that use event-time, the epoch mechanism introduces additional operations on the read/write paths. We provide a relatively efficient implementation in the PoC, and overall its overhead is relatively controllable.</span></p><h2 id="FLIP425:AsynchronousExecutionModel-OtherNon-recordEvents"><span class="ne-text">Other Non-record Events</span></h2><p><span class="ne-text"><strong>For other non-record events</strong>, other than watermark and checkpoint barrier, such as RecordAttributes, EndOfDataMarker and so on, they will leverage the EpochManager to handle the execution as well. By default, they hold <strong>strict order</strong> with subsequent and preceding records. Also, we won't expose the option for the execution order of each non-record event until we have identify the<span style="color: rgb(34,34,34);"><span> concrete use cases that necessitate it.</span></span></span></p><h2 id="FLIP425:AsynchronousExecutionModel-RecordContext"><span class="ne-text" style="color: rgb(0,0,0);">Record Context</span></h2><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">Within the task thread, operations of multiple records are executed in a time-sliced manner, thus context switching is performed. It is essential to maintain a context that preserves the necessary variables required by each operation. Thus the 'record context' is introduced. All operations for one record will share the same record context, which consists of at least 5 parts:</span></p><ol class="ne-ol"><li><span class="ne-text" style="color: rgb(0,0,0);">the record itself</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">the key group to which the record belongs</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">the hashcode of this record</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">the reference count of this record as section "Element Order" introduced.</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">the epoch this record belongs to as the section "Watermark" described.</span></li></ol><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">An operation and a record context constitute an execution unit, which will be encapsulated into a mail and delivered to the mailbox for execution.</span></p><p class="ne-p"><br/></p><h2 id="FLIP425:AsynchronousExecutionModel-FasterCheckpointDrain"><span class="ne-text" style="color: rgb(0,0,0);">Faster Checkpoint Drain</span></h2><p class="ne-p">The record processing but also enlarge the size of elements to drain during sync phase of checkpointing. Thus we consider<span style="color: rgb(34,34,34);"> an optimization here. Since the record processing has been broken down to state requests and its callback, we could drain all the running state requests and checkpoint the buffered ones. This will greatly shorten the drain time.</span></p><p class="ne-p"><span style="color: rgb(34,34,34);"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image image-center" draggable="false" width="501" src="https://cwiki.apache.org/confluence/download/attachments/293046859/drain.png?version=1&modificationDate=1711530317000&api=v2" data-image-src="https://cwiki.apache.org/confluence/download/attachments/293046859/drain.png?version=1&modificationDate=1711530317000&api=v2" data-unresolved-comment-count="0" data-linked-resource-id="298781339" data-linked-resource-version="1" data-linked-resource-type="attachment" data-linked-resource-default-alias="drain.png" data-base-url="https://cwiki.apache.org/confluence" data-linked-resource-content-type="image/png" data-linked-resource-container-id="293046859" data-linked-resource-container-version="50" alt=""></span></span></p><p class="ne-p"><span style="color: rgb(34,34,34);">We will consider a set of declarative APIs and a `declareProcess()` function that users should implement in some newly introduced AbstractStreamOperator, built on top of current execution model. User could declare each processing code block (callback) with an unique id. During a checkpoint, the buffered state requests and corresponding ids will be snapshoted, and when restoring, the restored state requests will find the corresponding code block by the restored id. The code block is declared in user code and will be loaded each time before task runs. After all state requests and their callbacks restored in AEC, the whole pipeline could be resumed.</span></p><p class="ne-p"><span style="color: rgb(34,34,34);">Note that we will provide the set of declarative APIs and a `declareProcess()` in upcoming FLIPs, <strong>the raw usage of `processElement` is strictly for testing purpose.</strong></span></p><h2 id="FLIP425:AsynchronousExecutionModel-ErrorHandling"><span class="ne-text">Error Handling</span></h2><p><span class="ne-text">The <a href="https://cwiki.apache.org/confluence/x/eZ2zDw" rel="nofollow">FLIP-368</a> proposed to reorganize the exceptions thrown by state interfaces. This FLIP also adopts the design from <a href="https://cwiki.apache.org/confluence/x/eZ2zDw" rel="nofollow">FLIP-368</a>, ensuring that all state interfaces throw unchecked exceptions and, consequently, do not declare any exceptions in their signatures. In cases where an exception occurs while accessing the state, the job should fail. This relieves the user from the burden of error handling, which is particularly cumbersome in asynchronous interfaces.</span></p><h2 id="FLIP425:AsynchronousExecutionModel-Configurations">Configurations</h2><p><span class="ne-text">A new set of configure options is introduced as well as their default value:</span></p><ul><li><span class="ne-text">execution.async-state.enabled: true</span></li><li><span class="ne-text">execution.async-state.in-flight-records-limit: 1000 ---- The record count that the AEC permit to proceed</span></li><li><span class="ne-text">execution.async-state.buffer-size: 1000 ---- The size of 'Active Buffer', usually equal to or smaller than the 'execution.async-state.in-flight-records-limit'</span></li><li><span class="ne-text">execution.async-state.buffer-timeout: 1s ---- The timeout for 'Active Buffer'</span></li></ul><p><span class="ne-text">The following options may be added when we identify <span style="color: rgb(34,34,34);"><span>concrete use cases:</span></span></span></p><ul><li><span class="ne-text">execution.async-state.order-with-same-key: Record-ordered (or state-ordered)</span></li><li><span class="ne-text">execution.async-state.event-order.watermark: Out-of-order (or Strictly-ordered)</span></li><li><span class="ne-text">execution.async-state.event-order.*: Strictly-ordered (or Out-of-order) —— for non-records other than watermark and checkpoint barrier</span><ul><li><span class="ne-text">More type will be added if we do need the out-of-order optimization for some else non-record events.</span></li></ul></li></ul><h1 id="FLIP425:AsynchronousExecutionModel-Performanceanalysis"><span class="ne-text">Performance analysis</span></h1><h2 id="FLIP425:AsynchronousExecutionModel-PoCTest"><span class="ne-text">PoC Test</span></h2><p class="ne-p"><span class="ne-text">A PoC (repo: <a class="external-link" href="https://github.com/ververica/flink-poc/tree/disagg-poc-2" rel="nofollow">https://github.com/ververica/flink-poc/tree/disagg-poc-2</a>) was developed to validate the performance, t<span style="color: rgb(31,31,31);">he PoC running instructions are provided in the <a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">appendix in FLIP-423</a> for those interested in exploring the setup further.</span></span></p><p class="ne-p"><strong><span class="ne-text">Environment Setup</span></strong><span class="ne-text">:</span></p><ul class="ne-ul"><li><strong><span class="ne-text" style="color: rgb(52,52,52);">Version: </span></strong><span class="ne-text" style="color: rgb(52,52,52);">based on</span><strong><span class="ne-text" style="color: rgb(52,52,52);"> </span></strong><span class="ne-text" style="color: rgb(52,52,52);">Flink 1.19</span></li><li><strong><span class="ne-text" style="color: rgb(52,52,52);">Deployment mode</span></strong><span class="ne-text" style="color: rgb(52,52,52);">: yarn per-Job</span></li><li><span style="color: rgb(52,52,52);"><strong>Flink yarn cluster</strong>:</span><ul class="ne-ul"><li><span class="ne-text" style="color: rgb(0,0,0);">1 master 2 workers</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">specifications: ecs.g7.2xlarge 8 vCPU 32 GiB (Alibaba Cloud)</span></li></ul></li><li><strong><span class="ne-text" style="color: rgb(0,0,0);">HDFS cluster</span></strong><span class="ne-text" style="color: rgb(0,0,0);">:</span><ul class="ne-ul"><li><span class="ne-text" style="color: rgb(0,0,0);">1 master 2 workers</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">specifications: ecs.g7.2xlarge 8 vCPU 32 GiB (Alibaba Cloud)</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">yarn cluster and HDFS cluster are on the same LAN</span></li></ul></li><li><strong><span class="ne-text" style="color: rgb(0,0,0);">State backend</span></strong><span class="ne-text" style="color: rgb(0,0,0);">: ForSt (Based on RocksDB 8.5.3)</span></li></ul><ul class="ne-ul"><li><strong><span class="ne-text">Job config</span></strong><span class="ne-text">:</span><ul class="ne-ul"><li><strong><span class="ne-text" style="color: rgb(52,52,52);">Memory</span></strong><span class="ne-text" style="color: rgb(52,52,52);">: Task Manager 4GB, Job Manager 4GB</span></li><li><strong><span class="ne-text" style="color: rgb(0,0,0);">checkpoint</span></strong><span class="ne-text" style="color: rgb(0,0,0);">: Disabled (Not fully supported in PoC yet)</span></li></ul></li></ul><ul class="ne-list-wrap"><ul class="ne-ul"><li><strong><span class="ne-text"><a class="external-link" href="https://github.com/ververica/flink-poc/tree/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcountasync" rel="nofollow">2 stages word count</a></span></strong><span class="ne-text">:</span></li></ul></ul><ul class="ne-list-wrap"><ul class="ne-list-wrap"><ul class="ne-ul"><li><span class="ne-text">Stage 1: populate different data to make the state size reach a certain size.</span></li><li><span class="ne-text">Stage 2: read/write the data from the key space of Stage 1, make the state size stable within a certain range. </span></li><li><span class="ne-text">Max TPS results are measured in Stage 2.</span></li><li><span class="ne-text"><a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/resources/submit-job.sh" rel="nofollow">Job parameter</a> && <a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/resources/test-flink-conf.yaml" rel="nofollow">configuration</a></span></li></ul></ul></ul><ul class="ne-list-wrap"><ul class="ne-ul"><li><span class="ne-text"><strong>Read/Write ratio</strong>: 1</span></li><li><span class="ne-text"><strong>RocksDB/ForSt block cache</strong>: 512 MB</span></li></ul></ul><p><br/></p><p><span class="ne-text"><strong>Max TPS Results(With Asynchronous Execution FLIP-425; Without IO Grouping FLP-426)</strong></span></p><div class="table-wrap"><table class="relative-table wrapped confluenceTable" style="width: 653.328px;"><colgroup><col style="width: 29.8594px;"/><col style="width: 285.312px;"/><col style="width: 147.469px;"/><col style="width: 85.8516px;"/><col style="width: 103.836px;"/></colgroup><tbody><tr><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text"><br/></span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text"><br/></span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">Local File Cache</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">TPS</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">State Size</span></p></td></tr><tr><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">1</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">Synchronous Execution + Local disk</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">-</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">17.1 K</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">5.52 G</span></p></td></tr><tr><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">2</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">Synchronous Execution + HDFS *</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">-</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">0.85 K</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">1 G *</span></p></td></tr><tr><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">3</span></p></td><td rowspan="2" style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">Asynchronous Execution + HDFS</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">0 GB (0)</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">7 K</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">5.8 G</span></p></td></tr><tr><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">4</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">♾️</span><span class="ne-text"><span> </span>GB (100%)</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">42.7 K</span></p></td><td style="text-align: left;" class="confluenceTd"><p class="ne-p"><span class="ne-text">5.8 G</span></p></td></tr></tbody></table></div><p><span style="color: rgb(122,134,154);"><em><span class="ne-text">*: Performance of </span><strong><span class="ne-text">Synchronous API + HDFS</span></strong><span class="ne-text"> is extremely poor, and it is hard to accumulate enough state size. The results are measured at smaller data size (1G instead of 5G)</span></em></span></p><p><span>Based on the PoC results,</span><span> we can draw several key conclusions regarding the performance implications:</span></p><ul><li><p><strong>DFS as primary storage exhibits more than 95% TPS decrease:<span> </span></strong><span>Utilizing DFS as the primary storage solution significantly hinders performance.</span><span> Compared to the baseline (Synchronous API + Local Disk using RocksDB),</span><span> disaggregated state storage without optimization exhibits<strong> </strong><span style="color: rgb(32,33,36);"><span style="color: rgb(31,31,31);">a </span>performance degradation exceeding 95%<span style="color: rgb(31,31,31);"> in terms of TPS.</span></span></span></p></li><li><p><strong>Asynchronous Execution Model exhibits 2.5x TPS improvements</strong><strong>:</strong><span> The introduction of asynchronous state APIs offers substantial performance improvements.</span><span><strong> </strong><span style="color: rgb(32,33,36);">In an extreme case with all data cached locally (line 4)</span>, <span style="color: rgb(32,33,36);"><span style="color: rgb(31,31,31);">this model achieves a </span>2.5x increase in TPS<span style="color: rgb(31,31,31);"> compared to the baseline.</span></span></span></p></li><li><p><strong>HDFS with Asynchronous Execution<span> </span><span style="color: rgb(31,31,31);">achieves<span> </span></span>40% of the baseline performance:<span> </span></strong><span style="color: rgb(31,31,31);">As an initial proof-of-concept (PoC) result (line 3),</span><strong><span style="color: rgb(31,31,31);"><span> </span></span></strong><span style="color: rgb(31,31,31);">asynchronous execution with HDFS without caching achieves<span> </span></span>40% of the baseline performance<span style="color: rgb(31,31,31);">.</span><span style="color: rgb(31,31,31);"><span> </span>We anticipate further improvements through optimizations like I/O grouping (<a href="https://cwiki.apache.org/confluence/display/FLINK/%5BWIP%5DFLIP-426%3A+Access+Remote+State+in+Batch" rel="nofollow">FLIP-426</a>) and leveraging the local disk as a secondary cache (FLIP-429).</span><span style="color: rgb(31,31,31);"><span> </span>These advancements aim to bring HDFS performance closer to that of the local disk setup.</span></p></li></ul><h2 id="FLIP425:AsynchronousExecutionModel-Per-recordoverhead"><span class="ne-text">Per-record overhead</span></h2><p class="ne-p"><span class="ne-text">As mentioned above, to ensure the correctness of data under asynchronous state API, we have introduced some changes for operators. Compared with synchronous execution, these changes have some overhead on critical paths:</span></p><ol class="ne-ol"><li><span class="ne-text">Epoch Manager: In order to ensure the semantics of the watermark, the </span><em><span class="ne-text">Epoch Manger</span></em><span class="ne-text"> numbers each input record to determine the association between the record and the watermark. Nonetheless, </span><em><span class="ne-text">Epoch Manager</span></em><span class="ne-text"> is only needed in asynchronous state access and event-time scenarios, and in theory, this overhead is relatively controllable.</span></li><li><span class="ne-text">Key Accounting Unit: All state accesses under each record need to be checked by the </span><em><span class="ne-text">Key Accounting Unit</span></em><span class="ne-text"> before execution, which will increase CPU overhead. When using </span><strong><span class="ne-text">Disaggregated State Storage</span></strong><span class="ne-text">, the bottleneck is generally in I/O rather than CPU, and the overhead caused by the </span><em><span class="ne-text">Key Accounting Unit</span></em><span class="ne-text"> is less likely to become a bottleneck.</span></li></ol><h2 id="FLIP425:AsynchronousExecutionModel-Strictly-orderedmodevs.Out-of-ordermodeforwatermarkprocessing"><span class="ne-text">Strictly-ordered mode vs. Out-of-order mode for watermark processing</span></h2><p><span class="ne-text">A performance test is also conducted evaluating how much enhancement we can achieve by enabling out-of-order for watermark processing. The test job is WordCount with a simple WatermarkStrategy, which code can be found <a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcountasync/EventTimeWordCount.java" rel="nofollow">here</a>. The environment setup is basically identical with the PoC test above except for the state size.</span></p><div class="table-wrap"><table class="relative-table wrapped confluenceTable" style="width: 71.5948%;"><colgroup><col style="width: 5.26313%;"/><col style="width: 22.2965%;"/><col style="width: 21.7228%;"/><col style="width: 30.2391%;"/><col style="width: 20.3827%;"/></colgroup><tbody><tr><th scope="col" class="confluenceTh"><br/></th><th scope="col" class="confluenceTh">State Size</th><th scope="col" class="confluenceTh">Local File Cache</th><th scope="col" class="confluenceTh">TPS for Strictly-ordered mode</th><th scope="col" class="confluenceTh">TPS for out-of-order mode</th></tr><tr><td class="confluenceTd">1</td><td class="confluenceTd">~140MB</td><td class="confluenceTd">0%</td><td class="confluenceTd">25.7 K</td><td class="confluenceTd">43.8 K <span style="color: rgb(0,255,0);"><strong> <span style="color: rgb(51,153,102);">+70%</span></strong></span></td></tr><tr><td class="confluenceTd">3</td><td class="confluenceTd">~5.6 GB</td><td class="confluenceTd">0%</td><td class="confluenceTd">5.6 K</td><td class="confluenceTd">6.5 K <strong><span style="color: rgb(0,128,0);">+16%</span></strong></td></tr></tbody></table></div><p>As we can see the out-of-order mode outperforms the strictly-ordered mode, the effect is more obvious when the CPU becomes the bottleneck. </p><h2 id="FLIP425:AsynchronousExecutionModel-SynchronousexecutionwithasynchronousAPIs"><span class="ne-text">Synchronous execution with asynchronous APIs</span></h2><p class="ne-p"><span class="ne-text">We adapt the existing states of the Hashmap state backend and RocksDB state backend to the asynchronous state API to test the overhead of the asynchronous state API.</span></p><p class="ne-p"><span class="ne-text">In detail, we wrap the original state and return a completed StateFuture. Taking "heap value state" as an example, the pseudo-code is as follows:</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">public class AsyncHeapValueState<K, N, SV> implements AsyncValueState<SV> {
private HeapValueState<K, N, SV> heapValueState;
public AsyncHeapValueState(HeapValueState<K, N, SV> heapValueState) {
this.heapValueState = heapValueState;
}
@Override
public StateFuture<SV> value() {
SV value = heapValueState.value();
StateFuture future = new SyncExeStateFuture<>();
future.complete(value);
return future;
}
@Override
public StateFuture<Void> update(SV value) {
heapValueState.update(value);
StateFuture future = new SyncExeStateFuture<>();
future.complete(null);
return future;
}
}</pre>
</div></div><p class="ne-p"><span class="ne-text">This approach requires maintaining additional </span><code class="ne-code"><span class="ne-text">StateFuture</span></code><span class="ne-text"> in the JVM heap, which increases CPU and memory overhead. 2 stage word count is used to test, and checkpoint is disabled, the performance comparison is as follows:</span></p><div class="table-wrap"><table class="wrapped confluenceTable"><tbody><tr><td class="confluenceTd"><br/></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">key number</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">heap size</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">state size</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">TPS</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Hashmap</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">1 M</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">1.35 GB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">39.3 MB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">974.5 K</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Hashmap with async API</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">1 M</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">1.48 GB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">39.3 MB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">925.3 K </span><strong><span class="ne-text" style="color: rgb(223,42,63);">-5%</span></strong></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">RocksDB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">40 M</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">1015 MB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">683 MB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">267 K</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">RocksDB with async API</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">40 M</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">1016 MB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">683 MB</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">256.4 K </span><strong><span class="ne-text" style="color: rgb(223,42,63);">-3.9%</span></strong></p></td></tr></tbody></table></div><p class="ne-p"><span class="ne-text">From the above table, the asynchronous APIs can bring ~5% TPS regression, and the impact on Hashmap state backend is more significant. More variables in java heap bring more JVM GC, while the Hashmap state backend requires more java heap memory than the RocksDB state backend.</span></p><p class="ne-p"><span class="ne-text"><strong>Profiling: </strong>For test 'Hashmap with async API' we managed to capture a <a href="https://cwiki.apache.org/confluence/download/attachments/293046859/heap.html.zip?version=1&modificationDate=1709708370000&api=v2" data-linked-resource-id="294816382" data-linked-resource-version="1" data-linked-resource-type="attachment" data-linked-resource-default-alias="heap.html.zip" data-nice-type="Zip Archive" data-linked-resource-content-type="application/zip" data-linked-resource-container-id="293046859" data-linked-resource-container-version="50">flame graph</a>, which reveals that the framework overhead (including reference counting, future-related code and so on) consumes about 9% of the keyed operator cpu time. We consider it acceptable given that it is a pure memory state access and the calculation is quite simple (only plus 1 for each record). The overhead will be negligible when involving slow I/O access and asynchronous execution.</span></p><h2 id="FLIP425:AsynchronousExecutionModel-ComparedwithBatch-styleAPIs"><span class="ne-text">Compared with Batch-style APIs</span></h2><p class="ne-p"><span class="ne-text">This FLIP proposes a future-style APIs for single key-value accessing, while another alternative is to provide batch-style APIs, leaving the data gathering work to user. The advantage of batch-style APIs is that users may have some prior knowledge that can help reduce the state access when batching. But batch-style APIs lack flexibility and are difficult to use. With future-style APIs, user can still buffer inputs in advance and process iteratively within a batch in their code. The potential upper limit of performance for batch-style APIs and asynchronous APIs have no much difference. Compared with batch-style APIs, user could write their code more easily since they deal with one record with future-style APIs.</span></p><h1 id="FLIP425:AsynchronousExecutionModel-Compatibility,Deprecation,andMigrationPlan">Compatibility, Deprecation, and Migration Plan</h1><p class="ne-p"><span class="ne-text">The newly introduced API is designed to coexist alongside the existing API. The code path for the new API is completely independent from the original one. We strongly recommend that users stick to either the new API or the original API exclusively, rather than mixing their usage. Although using both the asynchronous and synchronous APIs simutaneously won't cause compatibility issues, it may lead to suboptimal performance. The synchronous API can block the task thread until its execution is complete, potentially resulting in a performance regression when compared to using the asynchronous state calls exclusively.</span></p><p class="ne-p"><span class="ne-text">As a subsequent step following this FLIP, all SQL operators will be transitioned to access state asynchronously. Additional FLIP(s) will be proposed to facilitate this transition once the current FLIP has been fully integrated.</span></p><h1 id="FLIP425:AsynchronousExecutionModel-TestPlan">Test Plan</h1><p class="ne-p"><span class="ne-text">New UT/ITs will be introduced for each module described in the section "Proposed Changes". New E2E tests of jobs using asynchronous state APIs will also be delivered.</span></p><h1 id="FLIP425:AsynchronousExecutionModel-RejectedAlternatives">Rejected Alternatives</h1><p>Same as the "Rejected Alternatives" section in <span class="ne-text"><a href="https://cwiki.apache.org/confluence/x/SYp3EQ" rel="nofollow">FLIP-424</a>.</span></p><h1 id="FLIP425:AsynchronousExecutionModel-Appendix:"><span class="ne-text">Appendix: </span></h1><h3 id="FLIP425:AsynchronousExecutionModel-HowtorunSynchronousexecutionwithasynchronousAPIs"><span class="ne-text">How to run </span><span class="ne-text">Synchronous execution with asynchronous APIs</span></h3><p class="auto-cursor-target">The modifications about “Synchronous execution with asynchronous APIs“ are mainly in this <a class="external-link" href="https://github.com/ververica/flink-poc/commit/3783a7d824245950e351c40303bd4ba6477b943b" rel="nofollow">commit</a>, it allows you to use asynchronous APIs to manipulate the state of an existing state backend.</p><ol><li>If you don't care about the performance testing, <a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/WordCountTest.java#L102" rel="nofollow">this test</a> can directly run in the IDE.</li><li>If you are curious about the performance, it is recommended to run a job on a cluster.</li></ol><p><strong>For performance testing:</strong></p><p class="auto-cursor-target">Please refer to <span class="ne-text"><span style="color: rgb(31,31,31);"><a href="https://cwiki.apache.org/confluence/x/R4p3EQ" rel="nofollow">appendix in FLIP-423</a></span></span> for the previous steps about compiling.</p><p class="auto-cursor-target"><strong><span class="ne-text">Step 1: Update the state backend config</span></strong></p><ul style="list-style-type: square;"><li class="auto-cursor-target"><p class="auto-cursor-target">Modify the state backend in <a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/resources/test-flink-conf.yaml#L316" rel="nofollow">flink-conf.yaml</a> </p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">state.backend: hashmap # For Hashmap State backend
# state.backend: rocksdb # For RocksDB state backend</pre>
</div></div></li></ul><p><strong>Step 2: Write a job that uses asynchronous APIs, or you can directly use the <a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcountasync/WordCount.java#L70" rel="nofollow">word count</a> we provide.</strong></p><p><strong>Step 3: Run the job. This script(<a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/resources/submit-job.sh" rel="nofollow">submit-job.sh</a>) may help.</strong></p><p><br/></p><p><br/></p><p><br/></p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-425%3A+Asynchronous+Execution+Model">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=293046859&revisedVersion=50&originalVersion=49">View Changes Online</a>
</div>
</div>Yanfei Lei2024-02-23T02:53:30ZFLIP-423: Disaggregated State Storage and Management (Umbrella FLIP)Zakelly Lantag:cwiki.apache.org,2009:page-293046855-1022024-03-27T09:16:49Z2024-02-23T02:47:28Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~zakelly
">Zakelly Lan</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<p class="auto-cursor-target">by <a class="confluence-userlink user-mention" data-username="curcur" href="https://cwiki.apache.org/confluence/display/~curcur" data-linked-resource-id="128652457" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Yuan Mei</a> <a class="confluence-userlink user-mention" data-username="zakelly" href="https://cwiki.apache.org/confluence/display/~zakelly" data-linked-resource-id="181311105" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Zakelly Lan</a> <a class="confluence-userlink user-mention" data-username="lijinzhong" href="https://cwiki.apache.org/confluence/display/~lijinzhong" data-linked-resource-id="263427888" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Jinzhong Li</a> <a class="confluence-userlink user-mention" data-username="masteryhx" href="https://cwiki.apache.org/confluence/display/~masteryhx" data-linked-resource-id="181309527" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Hangxiang Yu</a> <a class="confluence-userlink user-mention" data-username="fredialei" href="https://cwiki.apache.org/confluence/display/~fredialei" data-linked-resource-id="211886241" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Yanfei Lei</a> <a class="confluence-userlink user-mention" data-username="fengwang" href="https://cwiki.apache.org/confluence/display/~fengwang" data-linked-resource-id="199533258" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Feng Wang</a></p><div class="table-wrap"><table class="wrapped confluenceTable" style="letter-spacing: 0.0px;"><colgroup><col/><col/></colgroup><tbody><tr><th class="confluenceTh">Discussion thread</th><td class="confluenceTd"><a class="external-link" href="https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0" rel="nofollow">https://lists.apache.org/thread/ct8smn6g9y0b8730z7rp9zfpnwmj8vf0</a></td></tr><tr><th colspan="1" class="confluenceTh">Vote thread</th><td colspan="1" class="confluenceTd"><br/></td></tr><tr><th class="confluenceTh">JIRA</th><td class="confluenceTd"><br/></td></tr><tr><th colspan="1" class="confluenceTh">Release</th><td colspan="1" class="confluenceTd">2.0 2.1</td></tr></tbody></table></div><p>Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).</p><p><style type='text/css'>/*<![CDATA[*/
div.rbtoc1711724893314 {padding: 0px;}
div.rbtoc1711724893314 ul {margin-left: 0px;}
div.rbtoc1711724893314 li {margin-left: 0px;padding-left: 0px;}
/*]]>*/</style><div class='toc-macro rbtoc1711724893314'>
<ul class='toc-indentation'>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-Motivation'>Motivation</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-Challenges'>Challenges</a>
<ul class='toc-indentation'>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-LocalDiskConstraints'>Local Disk Constraints</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-SpikyResourceUsage'>Spiky Resource Usage</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-ElasticityandFastRescaling'>Elasticity and Fast Rescaling</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-LightandFastCheckpoints'>Light and Fast Checkpoints</a></li>
</ul>
</li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-High-LevelOverviewandDesign'>High-Level Overview and Design</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-HurdlesfortheDisaggregatedStateModel'>Hurdles for the Disaggregated State Model</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-ProposedChanges'>Proposed Changes</a>
<ul class='toc-indentation'>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-Non-blockingAsynchronousExecutionModel:ParallelI/O(FLIP-425)'>Non-blocking Asynchronous Execution Model: Parallel I/O (FLIP-425)</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-OptimizedbuffercheckpointingforAsynchronousExecutionModel(FLIP-TBD)'>Optimized buffer checkpointing for Asynchronous Execution Model (FLIP-TBD)</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-BatchingforNetworkI/O:BeyondParallelI/O(FLIP-426)'>Batching for Network I/O: Beyond Parallel I/O (FLIP-426)</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-DisaggregatedStateStore:ForSt(FLIP-427)'>Disaggregated State Store: ForSt (FLIP-427)</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-FasterCheckpoint/Restore/Rescale:LeverageSharedDFS(FLIP-428)'>Faster Checkpoint/Restore/Rescale: Leverage Shared DFS (FLIP-428)</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-TieredStorage:LocalDiskasaSecondaryCache(FLIP-429)'>Tiered Storage: Local Disk as a Secondary Cache (FLIP-429)</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-RemoteCompaction(FLIP-430)'>Remote Compaction (FLIP-430)</a></li>
</ul>
</li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-PublicInterfaceChanges'>Public Interface Changes</a>
<ul class='toc-indentation'>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-StateBackend:ForSt'>StateBackend: ForSt</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-AsynchronousStateAPIs(FLIP-424)'>Asynchronous State APIs (FLIP-424)</a></li>
</ul>
</li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-PoCResults'>PoC Results</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-RoadMap+LaunchingPlan'>Road Map + Launching Plan</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-RejectedAlternatives'>Rejected Alternatives</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-References'>References</a></li>
<li><a href='#FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-Appendix:HowtorunthePoC'>Appendix: How to run the PoC</a></li>
</ul>
</div></p><p class="ne-p" style="text-align: left;"><span class="ne-text">One major advantage of Flink is its efficient and easy-to-use state management mechanism. However, this mechanism has not evolved much since it was born and hasn't kept pace with the demands of the cloud-native era. In this FLIP, we revisit the current architecture of Flink state management model and propose an augmented alternative in a disaggregated fashion.</span></p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-Motivation">Motivation</h1><p class="ne-p" style="text-align: left;"><span class="ne-text">The past decade has witnessed a dramatic shift in Flink's deployment mode, workload patterns, and hardware improvements. We've moved from the map-reduce era where workers are computation-storage tightly coupled nodes to a cloud-native world where containerized deployments on Kubernetes become standard. At the same time, hardware improvements, like the tenfold increase in network bandwidth (from mere hundreds of Mbps to 25Gbps, comparable to local SSD bps), unlock compelling options for state storage. Fast intranet transmission and low-cost object storage make cloud storage a feasible and cost-effective choice. Flink's workload scale has also dramatically transformed. A state size of several hundred megabytes, once considered large, pales in comparison to the multi-terabyte states commonly encountered in today's applications. </span></p><p class="ne-p" style="text-align: left;"><span class="ne-text">Given these significant changes in deployment, hardware, and workloads, the current state access model heavily reliant on local disks no longer aligns with Flink's future in the cloud-native era.</span></p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-Challenges"><span class="ne-text">Challenges </span></h1><p class="ne-p" style="text-align: left;"><span class="ne-text">We discuss key challenges faced in Flink state management today in this session, including those newly introduced by cloud-native demands, and those longstanding due to the limitations of the local storage model. We aim to resolve these challenges with the new design.</span></p><h4 style="text-align: left;" id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-LocalDiskConstraints"><strong><span class="ne-text">Local Disk Constraints</span></strong></h4><p class="ne-p" style="text-align: left;"><span class="ne-text">While containerization technology provides resource isolation and secure job operations, it comes at a cost: resource is pre-allocated. As a result, local disk space is fixed and limited and becomes a bottleneck for data-intensive jobs. In the current model, we do not have good solutions except scaling up the job if running out of disk space, which is very costly.</span></p><h4 style="text-align: left;" id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-SpikyResourceUsage"><strong><span class="ne-text">Spiky Resource Usage</span></strong></h4><p class="ne-p" style="text-align: left;"><span class="ne-text">The current state model triggers periodic CPU and network I/O bursts during checkpointing as shown in Figure 1. This is because checkpointing triggers rocksdb compaction (CPU spike) and state file uploads to DFS (network I/O spike). For large-state jobs, those spikes happen almost the same time across all tasks, leading to the instability of the entire cluster. This also results in data processing TPS hampered by performance-crippling spikes during checkpoints.</span></p><p class="ne-p" style="text-align: left;"><br/></p><p class="ne-p" style="text-align: center;"><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource" draggable="false" height="150" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/238920/1706263584365-c8ed432d-4c82-46a0-98f0-1ac4b8444edf.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/238920/1706263584365-c8ed432d-4c82-46a0-98f0-1ac4b8444edf.png"></span></p><p class="ne-p" style="text-align: center;"><span class="ne-text" style="color: rgb(122,134,154);"> Figure 1: CPU usage of Flink Job, periodic spikes incur during checkpointing</span></p><h4 style="text-align: left;" id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-ElasticityandFastRescaling"><span class="ne-text">Elasticity and </span><strong><span class="ne-text">Fast Rescaling</span></strong></h4><p class="ne-p" style="text-align: left;"><span class="ne-text">Achieving zero-downtime rescaling remains a challenge for Flink, particularly for those with large state sizes. The current model involves state redistribution, download and rebuild during rescaling, hindering even near-zero downtime goals. While <a class="external-link" href="https://issues.apache.org/jira/browse/FLINK-31238" rel="nofollow">FLINK-31238</a> reduces rebuild time by merging and clipping SST files directly, downloading large state files to local disks is still a bottleneck. This significantly extends rescaling durations and can even block downscaling attempts if the aggregate state size exceeds local disk capacity.</span></p><h4 style="text-align: left;" id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-LightandFastCheckpoints"><strong><span class="ne-text">Light and Fast Checkpoints</span></strong></h4><p class="ne-p" style="text-align: left;"><span class="ne-text">Checkpointing is a routine operation for Flink Engine to ensure fault tolerance and enable rescaling. Besides that, end-to-end exactly-once hinges on committing data in a granularity of each checkpoint, a fundamental requirement for streaming DB. Hence making light and fast checkpoints is crucial. While generic log-based checkpoiniting[] tackles the problem, it introduces an additional log layer to double-write state changes and incurs extra CPU and network overheads.</span></p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-High-LevelOverviewandDesign"><span class="ne-text">High-Level Overview and Design</span></h1><p style="text-align: left;"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource image-center" draggable="false" height="400" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/238920/1706264202357-d8b1ec7e-87dc-4265-8e46-dd7f15c3c03c.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/238920/1706264202357-d8b1ec7e-87dc-4265-8e46-dd7f15c3c03c.png"></span></p><p style="text-align: center;"><span style="color: rgb(122,134,154);">Figure 2: The Disaggregated Model of State Management</span></p><p class="ne-p"><span class="ne-text">In the current model, Flink operators read and write states through a component named State Backend as illustrated on the left-hand side in Figure 2. States are put on files on local disks if not fit in memory. As aforementioned, to recover after a failure or enable rescaling, states are flushed to disk and state files are dumped to a durable storage periodically to make checkpoints, and read back after recovery/rescaling. This model greatly relies on the capability of local disks, which leads to the challenges listed above.</span></p><p class="ne-p"><span class="ne-text">The right-hand side of Figure 2 illustrates the proposed new model. In the new model, local disks are optional as a secondary cache, and the remote DFS (e.g., S3/OSS/HDFS/GFS...) serves as a primary storage for states. Therefore, the remote DFS becomes the source of truth. This is a fundamental change from relying on local disks to a complete computation and storage disaggregated model. States are streamed to DFS continuously; memory and local disks act as a cache. This disaggregated architecture embraces a cloud-native approach and unlocks several advantages:</span></p><ol class="ne-ol"><li><span class="ne-text">Local disk is no longer a constraint as DFS used as the primary storage.</span></li><li><span class="ne-text">The checkpointing procedure is fast, completing within 10s regardless of state size. Continuous streaming of states to DFS significantly reduces the amount of data needed to write during the checkpointing procedure, leading to rapid completion.</span></li><li><span class="ne-text">DB compaction is decoupled from the checkpointing procedure and not bound to a specific Task Manager anymore. This allows for independent scheduling of compaction tasks, enabling better load-balancing and eliminating disruptive resource bursts cluster-wide.</span></li><li><span class="ne-text">As for recovery and rescaling, state files are instantly accessible after the State Backend is up, eliminating the need for time-consuming local disk downloads.</span></li></ol><p class="ne-p"><span class="ne-text">The disaggregated model also facilitates a clean architecture for state sharing and querying. Checkpoint files are feasible to be shared between operators to eliminate redundant storage. For state querying, file-level APIs are provided to enable straightforward querying that resembles normal tables without relying on backend details or state specifics. In this FLIP, we focus on the disaggregated model and how it integrates with the current Flink Engine. Sharing and querying will be explored in future FLIPs.</span></p><h1 class="ne-p" id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-HurdlesfortheDisaggregatedStateModel"><span class="ne-text">Hurdles for the Disaggregated State Model</span></h1><p class="ne-p"><span class="ne-text">Our initial Proof of Concept (PoC) results showed a significant performance drawback for the disaggregated model (state store using DFS as primary storage) compared to the current model (local disk storage). Without local disk caches, the disaggregated model achieved only 2-5% of the maximum TPS observed with local disks across various setups. Analysis excluding page cache effects reveals that the main bottleneck lies in the blocking of the main task thread during state access as shown in Figure 3, making Flink engine performance highly sensitive to state access latency.</span></p><p class="ne-p" style="text-align: center;"><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource" draggable="false" height="400" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/238920/1706763190127-2b7d68d3-c303-4255-9400-362dd2fcc26d.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/238920/1706763190127-2b7d68d3-c303-4255-9400-362dd2fcc26d.png"></span></p><p class="ne-p" style="text-align: center;"><span style="color: rgb(122,134,154);">Figure 3: Current state access model in task thread</span></p><p class="ne-p" style="text-align: left;"><span class="ne-text">State access operates at a </span><strong><span class="ne-text">per-record</span></strong><span class="ne-text"> granularity, executed by the Flink task thread. These accesses are queued and processed sequentially within the mailbox executor. State writing is non-blocking, as writes are appended to a write buffer and asynchronously flushed to disk when full. However, state reading involves fetching data from the disk on cache misses, which causes the task thread to block and halt further processing until the data is retrieved. As illustrated below, HDFS has 20+ times higher latency than local disk (1.5ms Vs. 68us), which explains why TPS drops 95% when switching from local disk to HDFS.</span></p><p class="ne-p" style="text-align: center;"><span class="confluence-embedded-file-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource" draggable="false" height="150" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/238920/1706678085271-c191327d-bbe8-4c56-ac81-7e29088b2b7d.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/238920/1706678085271-c191327d-bbe8-4c56-ac81-7e29088b2b7d.png"></span></p><p class="ne-p" style="text-align: center;"><span style="color: rgb(122,134,154);">Table 1: Access Latency across different storage medium</span></p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-ProposedChanges"><span class="ne-text">Proposed Changes</span></h1><p class="ne-p"><span class="ne-text">Unlocking the disaggregated model requires tackling three key engine challenges:</span></p><ol class="ne-ol"><li><span class="ne-text">Implementing a </span><strong><span class="ne-text">non-blocking execution model</span></strong><span class="ne-text">: This unleashes full CPU potential while waiting for I/O or network operations.</span></li><li><span class="ne-text">Developing a </span><strong><span class="ne-text">disaggregated data store</span></strong><span class="ne-text">: This store must seamlessly handle both writing and reading from DFSs.</span></li><li><strong><span class="ne-text">Integrating with Flink Engine</span></strong><span class="ne-text">: The data store needs to be integrated with the Flink Engine and existing state management mechanism (checkpoint/recover/rescale).</span></li></ol><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-Non-blockingAsynchronousExecutionModel:ParallelI/O(FLIP-425)"><a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow"><span class="ne-text">Non-blocking Asynchronous Execution Model: </span><span class="ne-text" style="color: rgb(31,31,31);">Parallel I/O</span><span class="ne-text"> (FLIP-425)</span></a></h3><p class="ne-p"><span class="ne-text">In <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a>, we propose a non-blocking execution model enabling asynchronous state access. In this new model, <a href="https://cwiki.apache.org/confluence/x/SYp3EQ" rel="nofollow">asynchronous State APIs(FLIP-424)</a> are introduced. These APIs register callbacks with the task thread mailbox executor and allow for efficient chained execution. However, asynchronous access presents several new challenges:</span></p><ol class="ne-ol"><li><strong><span class="ne-text">Preserving Processing Order for Identical Keys:</span></strong><span class="ne-text"> Asynchronous execution does not inherently guarantee processing order for input data. This isn't an issue for input with different keys, as Flink's distributed nature already allows for some order variation between them. However, ensuring records with identical keys are processed in their arrival order is crucial for data integrity.</span></li><li><strong><span class="ne-text">Checkpointing Synchronization:</span></strong><span class="ne-text"> Asynchronous execution adds more complexities to the checkpoint synchronization phase. Before triggering state snapshots, both ongoing callback executions and buffered input meant to maintain processing order must be drained to ensure a consistent state capture.</span></li><li><strong><span class="ne-text" style="color: rgb(31,31,31);">Watermark and Timer:</span></strong><span class="ne-text" style="color: rgb(31,31,31);"> Watermark calculation and Timer triggering implicitly rely on order among in-flight data, </span><span class="ne-text" style="color: rgb(0,0,0);">necessitating</span><span class="ne-text" style="color: rgb(31,31,31);"> </span><span class="ne-text" style="color: rgb(0,0,0);">special handling</span><span class="ne-text" style="color: rgb(31,31,31);"> to </span><span class="ne-text" style="color: rgb(0,0,0);">guarantee</span><span class="ne-text" style="color: rgb(31,31,31);"> the </span><span class="ne-text" style="color: rgb(0,0,0);">intended </span><span class="ne-text" style="color: rgb(31,31,31);">semantics within the </span><span class="ne-text" style="color: rgb(0,0,0);">asynchronous</span><span class="ne-text" style="color: rgb(31,31,31);"> context.</span></li></ol><p class="ne-p"><span class="ne-text" style="color: rgb(31,31,31);"><a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a> introduces a new component named "Asynchronous Execution Controller(AEC)" to address these challenges and orchestrate the asynchronous execution. A new set of asynchronous state APIs is also introduced accordingly within this asynchronous paradigm. Please refer to <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a> for a deeper dive.</span></p><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-OptimizedbuffercheckpointingforAsynchronousExecutionModel(FLIP-TBD)">Optimized buffer checkpointing for Asynchronous Execution Model (FLIP-TBD)</h3><p class="ne-p">The asynchronous execution model introduced by FLIP-425 parallelize the record processing but also enlarge the size of elements to drain during sync phase of checkpointing. Thus we consider<span style="color: rgb(34,34,34);"> a set of declarative APIs and a `declareProcess()` function that users should implement in some newly introduced AbstractStreamOperator, we could get the declaration of record processing in runtime, broken down to requests and callbacks (lambdas) like FLIP-424 introduced. Thus we avoid the problem of lambda (de)serialization and instead we retrieve callbacks every time before a task runs. The next step is to provide an API allowing users to assign an unique id to each state request and callback, or automatically assign one by declaration order. Thus we can find the corresponding callback in runtime for each restored state request based on the id, then the whole pipeline can be resumed.</span></p><p class="ne-p"><span style="color: rgb(34,34,34);">More details will be added in following future FLIPs.</span></p><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-BatchingforNetworkI/O:BeyondParallelI/O(FLIP-426)"><a href="https://cwiki.apache.org/confluence/x/TYp3EQ" rel="nofollow"><span class="ne-text">Batching for Network I/O: </span><span class="ne-text" style="color: rgb(31,31,31);">Beyond Parallel I/O</span><span class="ne-text"> (FLIP-426)</span></a></h3><p class="ne-p" style="text-align: left;"><span class="ne-text">The asynchronous model introduced in <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a> effectively helps boost overall throughput with parallel I/O operations. However, simply expanding I/O parallelism is not enough for remote storage. For network I/O, the substantial round-trip time of RPC calls far outweighs the individual I/O size's impact. Hence, how to efficiently merge I/O requests into single network I/O operations and fetching multiple keys within a single call plays an important role in reducing the overhead of individual interactions and leverages available bandwidth more effectively. This becomes critically important for large-state jobs, where frequent state data fetching can lead to bottlenecks.</span></p><p class="ne-p" style="text-align: left;"><span class="ne-text"><a href="https://cwiki.apache.org/confluence/x/TYp3EQ" rel="nofollow">FLIP-426</a> solves this problem by:</span></p><ol class="ne-ol"><li><strong><span class="ne-text">Categorizing state operations:</span></strong><span class="ne-text"> This allows for fine-grained management and optimization of different types of data access.</span></li><li><strong><span class="ne-text">Grouping I/Os together:</span></strong><span class="ne-text"> Batching adjacent requests reduces individual RPC overhead and leverages network efficiency.</span></li><li><strong><span class="ne-text">Integrating with the "Asynchronous Execution Controller":</span></strong><span class="ne-text"> Integrating with <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a>'s orchestration mechanism for asynchronous execution.</span></li></ol><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-DisaggregatedStateStore:ForSt(FLIP-427)"><span class="ne-text"><a href="https://cwiki.apache.org/confluence/x/T4p3EQ" rel="nofollow">Disaggregated State Store: ForSt (FLIP-427)</a></span></h3><p class="ne-p"><span class="ne-text">Figure 2 illustrates our adopted approach of using an embedded key-value (k-v) store to complement remote storage. We introduce </span><strong><span class="ne-text">ForSt</span></strong><span class="ne-text">, a disaggregated version of frocksdb (the default local disk state store for Flink) to achieve this purpose. </span></p><p class="ne-p"><span class="ne-text">One alternative is using an independent k-v service like HBase. However, this approach introduces new dependencies on external services, which significantly complicates deployment, </span><span class="ne-text" style="color: rgb(31,31,31);">maintenance</span><span class="ne-text"> and management. Currently, Flink solely depends on Kubernetes (K8S) and Distributed File System (DFS) for high availability and checkpointing respectively. Ease of use and deployment is critical for open-source projects and we want to align with </span><span class="ne-text" style="color: rgb(31,31,31);">this philosophy</span><span class="ne-text">.</span></p><p class="ne-p"><span class="ne-text">While several research projects explore disaggregated, embedded key-value stores (like those referenced in disaggregated RocksDB[1] and RocksDB-Cloud[2]), no widely adopted, open-source solutions exist yet. By carefully weighing usability, extensibility, complexity, and performance as well as the efforts to integrate with the Flink engine, we decided to build a disaggregated state store named </span><strong><span class="ne-text">ForSt</span></strong><span class="ne-text"> on top of frocksdb. </span><span class="ne-text" style="color: rgb(31,31,31);">Additionally, we created a unified file system JNI proxy that leverages existing file system implementations in Flink ensuring compatibility with various file system options.</span></p><p class="ne-p"><span class="ne-text">Detailed design can be found in <a href="https://cwiki.apache.org/confluence/x/T4p3EQ" rel="nofollow">FLIP-427</a> and PoC implementation can be found at <a class="external-link" href="https://github.com/ververica/ForSt/tree/disagg-poc" rel="nofollow">https://github.com/ververica/ForSt/tree/disagg-poc</a>.</span></p><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-FasterCheckpoint/Restore/Rescale:LeverageSharedDFS(FLIP-428)"><span class="ne-text"><a href="https://cwiki.apache.org/confluence/x/UYp3EQ" rel="nofollow">Faster Checkpoint/Restore/Rescale: Leverage Shared DFS (FLIP-428)</a></span></h3><p class="ne-p" style="text-align: left;"><span class="ne-text" style="color: rgb(51,51,51);">By moving primary state storage to the remote filesystem (DFS) and using local disks optionally for caching, </span><span class="ne-text">ForSt</span><span class="ne-text" style="color: rgb(51,51,51);"> offers a tiered storage system for Flink.</span><span class="ne-text" style="color: rgb(0,0,0);"> </span><span class="ne-text" style="color: rgb(51,51,51);">This approach provides significant improvements for checkpointing, restoring, and rescaling </span><span class="ne-text" style="color: rgb(0,0,0);">by leveraging the fact that working state and checkpoint files share the same underlying file system.</span></p><ul class="ne-ul"><li><strong><span class="ne-text" style="color: rgb(0,0,0);">Fast Checkpoint:</span></strong><span class="ne-text" style="color: rgb(0,0,0);"> </span><span class="ne-text" style="color: rgb(51,51,51);">Since most state files already reside in DFS, only small incremental updates need to be uploaded during checkpointing, drastically reducing network transfer time.</span><span class="ne-text" style="color: rgb(0,0,0);"> In addition, both working state and checkpoints can reference the same underlying physical files, eliminating duplication. This saves storage space and further accelerates checkpoints.</span></li><li><strong><span class="ne-text" style="color: rgb(51,51,51);">Restore:</span></strong><span class="ne-text" style="color: rgb(51,51,51);"> With DFS plays as the primary storage, </span><span class="ne-text" style="color: rgb(51,51,51);">downloading large state files to local disks is avoided, significantly reducing restore time. Local disks (cache) can be gradually warmed up after the job starts, further optimizing performance.</span></li><li><strong><span class="ne-text" style="color: rgb(51,51,51);">Rescale</span></strong><span class="ne-text">: </span><span class="ne-text" style="color: rgb(51,51,51);">Rescaling leverages existing solutions like ClipDB/IngestDB to accelerate rebuilding the state store on DFS directly. Notice that since file downloads are eliminated, local disk constraints for downscaling are no longer an issue.</span></li></ul><p class="ne-p"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource image-center" draggable="false" width="650" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1706622488293-5fe806e6-d2b4-4bd2-8bcd-74c1698b1955.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/218902/1706622488293-5fe806e6-d2b4-4bd2-8bcd-74c1698b1955.png"></span></p><p class="ne-p" style="text-align: center;"><span class="ne-text" style="color: rgb(122,134,154);">Figure 4: Checkpoint/Restore/Rescale Mechanism (Current Model on the left; Disaggregated Model on the right)</span></p><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">For more detailed information on sharing the underlying file system between checkpoints and working states, along with changes to checkpointing, restoring, and rescaling procedures, please refer to <a href="https://cwiki.apache.org/confluence/x/UYp3EQ" rel="nofollow">FLIP-428</a>.</span></p><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-TieredStorage:LocalDiskasaSecondaryCache(FLIP-429)"><span class="ne-text" style="color: rgb(51,51,51);">Tiered Storage: Local Disk as a Secondary Cache (FLIP-429)</span></h3><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">While the asynchronous execution model (<a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a>) and network I/O grouping (<a href="https://cwiki.apache.org/confluence/x/TYp3EQ" rel="nofollow">FLIP-426</a>) have significantly enhanced performance, access latency in Table 1 </span><span class="ne-text" style="color: rgb(31,31,31);">reveal</span><span class="ne-text" style="color: rgb(51,51,51);"> that direct access to remote storage (DFS) remains 95% slower than local disk on average. Therefore, efficiently utilizing available local disks is crucial </span><span class="ne-text" style="color: rgb(31,31,31);">to maximize overall performance</span><span class="ne-text" style="color: rgb(51,51,51);">. In this context, local disks act as an optional secondary cache.</span><span class="ne-text"> </span></p><p class="ne-p"><span class="ne-text" style="color: rgb(51,51,51);">We propose two distinct disk-based caching solutions for disaggregated state management:</span></p><p class="ne-p"><strong><span class="ne-text">File-based Cache</span></strong><strong><span class="ne-text" style="color: rgb(51,51,51);">: </span></strong><span class="ne-text" style="color: rgb(51,51,51);">This approach is straightforward and </span><span class="ne-text" style="color: rgb(0,0,0);">structures the cache at the granular level of individual files</span><span class="ne-text" style="color: rgb(51,51,51);">. </span><span class="ne-text" style="color: rgb(0,0,0);">It replicates remote SST files onto local disks, enabling direct reading from local storage when the corresponding file resides therein, circumventing costly DFS round trip. While this method is commendable for its ease of implementation, its limited granularity may not be optimal for access patterns requiring finer differentiation.</span></p><p class="ne-p"><strong><span class="ne-text">Hybrid Block Cache: </span></strong><span class="ne-text" style="color: rgb(0,0,0);">This solution organizes the cache on disk at the block level, mirroring the granularity of the in-memory block cache. This facilitates unified management for both memory (Tier 1 Cache) and local disk (Tier 2 Cache) as a single entity. By ensuring that each block resides exclusively on either disk or memory (unlike the file-based cache), this approach minimizes duplication and likely experiences higher cache hit rates.</span></p><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">Initial PoC tests indicate that the hybrid block cache can provide an 80% increase in performance compared to the file-based cache for I/O-intensive jobs, both utilizing an LRU eviction policy. However, the inherent simplicity of the file-based cache necessitates less CPU overhead for management, particularly when I/O does not constitute the bottleneck. Therefore, it is better suited for scenarios where the state size is significantly smaller than the local storage capacity.</span></p><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">To optimize CPU efficiency across diverse scenarios, we are actively exploring an </span><strong><span class="ne-text" style="color: rgb(0,0,0);">"Adaptive Local Cache"</span></strong><span class="ne-text" style="color: rgb(0,0,0);"> capable of intelligently transitioning between the aforementioned caching solutions based on workload characteristics. As depicted in Figure 5, this solution aims to achieve optimal performance regardless of the prevailing conditions. Initial testing shows that </span><span class="ne-text" style="color: rgb(51,51,51);">with an adaptive local cache, we can achieve at least the same performance while states can fit into the local disk. More details will be revealed in future FLIP(s).</span></p><p class="ne-p"><span class="confluence-embedded-file-wrapper image-center-wrapper confluence-embedded-manual-size"><img class="confluence-embedded-image ne-image confluence-external-resource image-center" draggable="false" width="600" src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706510935957-1bd31d97-78ba-41d2-b474-bf2d75ccff0b.png" data-image-src="https://intranetproxy.alipay.com/skylark/lark/0/2024/png/116346/1706510935957-1bd31d97-78ba-41d2-b474-bf2d75ccff0b.png"></span></p><p class="ne-p" style="text-align: center;"><span style="color: rgb(122,134,154);">Figure 5: Adaptive Local Cache</span></p><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-RemoteCompaction(FLIP-430)"><span class="ne-text">Remote Compaction (FLIP-430)</span></h3><p class="ne-p" style="text-align: left;"><span class="ne-text" style="color: rgb(0,0,0);">Compaction is at the heart of LSM-based storage to enable competitive read performance and reduce space amplification, </span><span class="ne-text" style="color: rgb(0,0,0);">but it introduces bursts of high I/O and CPU usage. This poses a challenge in pre-allocated resource scenarios, potentially leading to underutilization or performance hiccups.</span></p><p class="ne-p" style="text-align: left;"><strong><span class="ne-text" style="color: rgb(0,0,0);">Remote Compaction</span></strong><span class="ne-text" style="color: rgb(0,0,0);"> addresses this issue by delegating compaction tasks to a separate work node from the LSM engine itself, as seen in RocksDB-cloud/Rockset [2]. This decouples the sporadic resource demands of compaction from the normal data process, leading to:</span></p><ul class="ne-ul"><li><strong><span class="ne-text" style="color: rgb(0,0,0);">More efficient resource utilization:</span></strong><span class="ne-text" style="color: rgb(0,0,0);"> dedicated nodes with pre-allocated resources specifically handle compaction across tasks, eliminating underutilization and ensuring smooth data processing.</span></li><li><strong><span class="ne-text" style="color: rgb(0,0,0);">Improved performance:</span></strong><span class="ne-text" style="color: rgb(0,0,0);"> By isolating compaction, the main LSM engine experiences minimal performance disruptions.</span></li></ul><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">The specific design details of the Remote Compaction architecture are still under development and will be finalized in future design documents.</span></p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-PublicInterfaceChanges"><span class="ne-text">Public Interface Changes</span></h1><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">This section details the steps involved in enabling disaggregated state management from a user perspective and related public interface changes.</span></p><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-StateBackend:ForSt"><span class="ne-text">StateBackend: ForSt</span></h3><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">We introduce a new state backend "ForSt" (named "For Stream") to manage disaggregated states. Existing state backends, "rocksdb" utilizing local disk and "heap" residing in memory, remain unaltered. Users can use them as before. To leverage the "ForSt" state backend, users need to configure the backend and additionally specify a working directory for states.</span></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeHeader panelHeader pdl" style="border-bottom-width: 1px;"><b>Configure to use ForSt</b></div><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: yml; gutter: false; theme: Default" data-theme="Default">state.backend.type: forst
# specify another storage other than the checkpoint storage (optional).
state.backend.forst.working-dir: hdfs:///path-to-your-storage</pre>
</div></div><p class="ne-p"><code><br/></code></p><h3 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-AsynchronousStateAPIs(FLIP-424)"><span class="ne-text" style="color: rgb(0,0,0);"><a href="https://cwiki.apache.org/confluence/x/SYp3EQ" rel="nofollow">Asynchronous State APIs (FLIP-424) </a></span></h3><p class="ne-p"><span class="ne-text" style="color: rgb(0,0,0);">As aforementioned, a new suite of <a href="https://cwiki.apache.org/confluence/x/SYp3EQ" rel="nofollow">asynchronous state APIs (FLIP-424)</a> has been introduced to seamlessly integrate with the <a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">Asynchronous Execution Model (FLIP-425)</a>. Existing synchronous state APIs remain unaltered, ensuring a smooth transition for users. <span style="color: rgb(31,31,31);">Importantly,</span><span style="color: rgb(31,31,31);"> as demonstrated in the PoC results within <span class="ne-text" style="color: rgb(0,0,0);"><a href="https://cwiki.apache.org/confluence/x/S4p3EQ" rel="nofollow">FLIP-425</a></span>,</span><span style="color: rgb(31,31,31);"> mimicking synchronous behavior using the asynchronous model incurs <span class="ne-text" style="color: rgb(0,0,0);">negligible</span> overhead (~3%).</span> </span></p><p class="ne-p"><span class="ne-text">How to use these asynchronous state APIs?</span></p><p><span class="ne-text"><strong style="text-align: left;">Table API/SQL Users</strong>: Don't need to make changes to their scripts since the SQL operators hide the details for them. A new set of SQL operators based on the asynchronous APIs will be provided later.</span></p><p><strong><span class="ne-text">For DataStream users</span></strong><span class="ne-text">: Need to write async operators based on async APIs. Here is an example of writing streaming word count (both in synchronous and asynchronous APIs) :</span></p><div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p>Such usage in the following example code in<span> </span><code class="c-mrkdwn__code">processElement/flatMap</code><span> </span>is an intermediate step. And as such it will be used for development and PoC purposes only. The long term solution will be discussed later.</p></div></div><p><br/></p><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">public class WordCount {
// Job definition
public static void main(String[] args) throws Exception {
// Some initialization code...
// Source
DataStream<String> text = env.fromSource(builder.build(), WatermarkStrategy.noWatermarks(), "file-input");
// Aggregate
DataStream<Tuple2<String, Integer>> counts =
text.flatMap(new Tokenizer())
.name("tokenizer")
.keyBy(value -> value.f0)
// User codes
.flatMap(new Counter())
.name("counter");
// Sink
counts.print().name("print-sink");
env.execute("WordCount");
}
// Core processing user function
public static final class Counter extends RichFlatMapFunction<Tuple2<String, Integer>, Tuple2<String, Integer>> {
// Initialization omitted.
private transient ValueState<Integer> wordCounter;
// Synchronous state API usage
@Override
public void flatMap(Tuple2<String, Integer> value, Collector<Tuple2<String, Integer>> out) throws Exception {
Integer val = wordCounter.value();
int updated = (val == null ? 1 : val + 1);
wordCounter.update(updated);
out.collect(Tuple2.of(value.f0, updated));
}
// Asynchronous state API usage
@Override
public void flatMap(Tuple2<String, Integer> value, Collector<Tuple2<String, Integer>> out) throws Exception {
AtomicInteger updated = new AtomicInteger(0);
wordCounter.value().thenCompose(
val -> {
if (val == null) {
updated.set(1);
} else {
updated.set(val + 1);
}
return wordCounter.update(updated.get());
}
).thenAccept(
empty -> {
out.collect(Tuple2.of(value.f0, updated.get()));
}
);
}
}
}</pre>
</div></div><p><br/><span style="color: rgb(31,31,31);">DataStream API users transitioning to asynchronous state APIs will find the code structure largely unchanged,</span><span style="color: rgb(31,31,31);"> with the primary differences lying in state initialization and the use of future-based asynchronous APIs for processing.</span><span style="color: rgb(31,31,31);"> For comprehensive details please refer to <span class="ne-text" style="color: rgb(0,0,0);"><a href="https://cwiki.apache.org/confluence/x/SYp3EQ" rel="nofollow">FLIP-424</a></span>.</span></p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-PoCResults"><span class="ne-text">PoC Results</span></h1><p class="ne-p"><br/></p><p class="ne-p"><strong><span class="ne-text">Environment Setup</span></strong><span class="ne-text">:</span></p><ul class="ne-ul"><li><strong><span class="ne-text" style="color: rgb(52,52,52);">Version: </span></strong><span class="ne-text" style="color: rgb(52,52,52);">based on</span><strong><span class="ne-text" style="color: rgb(52,52,52);"> </span></strong><span class="ne-text" style="color: rgb(52,52,52);">Flink 1.19</span></li><li><strong><span class="ne-text" style="color: rgb(52,52,52);">Deployment mode</span></strong><span class="ne-text" style="color: rgb(52,52,52);">: yarn per-Job</span></li><li><span style="color: rgb(52,52,52);"><strong>Flink yarn cluster</strong>:</span><ul class="ne-ul"><li><span class="ne-text" style="color: rgb(0,0,0);">1 master 2 workers</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">specifications: ecs.g7.2xlarge 8 vCPU 32 GiB (Alibaba Cloud)</span></li></ul></li><li><strong><span class="ne-text" style="color: rgb(0,0,0);">HDFS cluster</span></strong><span class="ne-text" style="color: rgb(0,0,0);">:</span><ul class="ne-ul"><li><span class="ne-text" style="color: rgb(0,0,0);">1 master 2 workers</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">specifications: ecs.g7.2xlarge 8 vCPU 32 GiB (Alibaba Cloud)</span></li><li><span class="ne-text" style="color: rgb(0,0,0);">yarn cluster and HDFS cluster are on the same LAN</span></li></ul></li><li><strong><span class="ne-text" style="color: rgb(0,0,0);">State backend</span></strong><span class="ne-text" style="color: rgb(0,0,0);">: ForSt (Based on RocksDB 8.5.3)</span></li></ul><ul class="ne-ul"><li><strong><span class="ne-text">Job config</span></strong><span class="ne-text">:</span><ul class="ne-ul"><li><strong><span class="ne-text" style="color: rgb(52,52,52);">Memory</span></strong><span class="ne-text" style="color: rgb(52,52,52);">: Task Manager 4GB, Job Manager 4GB</span></li><li><strong><span class="ne-text" style="color: rgb(0,0,0);">Checkpoint</span></strong><span class="ne-text" style="color: rgb(0,0,0);">: Disabled (Not fully supported in PoC yet)</span></li></ul></li></ul><ul class="ne-list-wrap"><ul class="ne-ul"><li><strong><span class="ne-text"><a class="external-link" href="https://github.com/ververica/flink-poc/tree/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcountasync" rel="nofollow">Two-stage word count</a></span></strong><span class="ne-text">:</span></li></ul></ul><ul class="ne-list-wrap"><ul class="ne-list-wrap"><ul class="ne-ul"><li><span class="ne-text">Stage 1: populate different data to make the state size reach a certain size.</span></li><li><span class="ne-text">Stage 2: read/write the data from the key space of Stage 1, make the state size stable within a certain range. </span></li><li><span class="ne-text">Max TPS results are measured in Stage 2.</span></li><li><span class="ne-text"><a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/resources/submit-job.sh" rel="nofollow">Job parameter</a> && <a class="external-link" href="https://github.com/ververica/flink-poc/blob/disagg-poc-2/flink-examples/flink-examples-streaming/src/main/resources/test-flink-conf.yaml" rel="nofollow">configuration</a></span></li></ul></ul></ul><ul class="ne-list-wrap"><ul class="ne-ul"><li><span class="ne-text"><strong>Read/Write ratio</strong>: 1</span></li><li><span class="ne-text"><strong>RocksDB/ForSt block cache</strong>: 512 MB</span></li></ul></ul><p><br/></p><p><span class="ne-text"><strong>Max TPS Results(With Asynchronous Execution FLIP-425; Without IO Grouping FLP-426)</strong></span></p><div class="table-wrap"><table class="relative-table wrapped confluenceTable" style="width: 61.9274%;"><colgroup><col style="width: 3.96596%;"/><col style="width: 44.0213%;"/><col style="width: 22.7382%;"/><col style="width: 13.2199%;"/><col style="width: 15.996%;"/></colgroup><tbody><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text"><br/></span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text"><br/></span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Local File Cache</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">TPS</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">State Size</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">1</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Synchronous Execution + Local disk</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">-</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">17.1 K</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">5.52 G</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">2</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">Synchronous Execution + HDFS *</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">-</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">0.85 K</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">1 G *</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">3</span></p></td><td rowspan="2" class="confluenceTd"><p class="ne-p"><span class="ne-text">Asynchronous Execution + HDFS</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">0 GB (0)</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">7 K</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">5.8 G</span></p></td></tr><tr><td class="confluenceTd"><p class="ne-p"><span class="ne-text">4</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">♾️</span><span class="ne-text"> GB (100%)</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">42.7 K</span></p></td><td class="confluenceTd"><p class="ne-p"><span class="ne-text">5.8 G</span></p></td></tr></tbody></table></div><p><span class="ne-text">*: Performance of </span><strong><span class="ne-text">Synchronous API + HDFS</span></strong><span class="ne-text"> is extremely poor, and it is hard to accumulate enough state size. The results are measured at smaller data size (1G instead of 5G)</span></p><p><span>Based on the PoC results,</span><span> we can draw several key conclusions regarding the performance implications:</span></p><ul><li><p><strong>DFS as primary storage exhibits more than 95% TPS decrease: </strong><span>Utilizing DFS as the primary storage solution significantly hinders performance.</span><span> Compared to the baseline (Synchronous API + Local Disk using RocksDB),</span><span> disaggregated state storage without optimization exhibits<strong> </strong><span style="color: rgb(32,33,36);"><span style="color: rgb(31,31,31);">a </span>performance degradation exceeding 95%<span style="color: rgb(31,31,31);"> in terms of TPS.</span></span></span></p></li><li><p><strong>Asynchronous Execution Model exhibits 2.5x TPS improvements</strong><strong>:</strong><span> The introduction of asynchronous state APIs offers substantial performance improvements.</span><span><strong> </strong><span style="color: rgb(32,33,36);">In an extreme case with all data cached locally (line 4)</span>, <span style="color: rgb(32,33,36);"><span style="color: rgb(31,31,31);">this model achieves a </span>2.5x increase in TPS<span style="color: rgb(31,31,31);"> compared to the baseline.</span></span></span></p></li><li><p><strong><strong>HDFS with Asynchronous Execution <span style="color: rgb(31,31,31);">achieves </span>40% of the baseline performance</strong>: </strong><span style="color: rgb(31,31,31);">As an initial proof-of-concept (PoC) result (line 3),</span><strong><span style="color: rgb(31,31,31);"> </span></strong><span style="color: rgb(31,31,31);">asynchronous execution with HDFS without caching achieves </span>40% of the baseline performance<span style="color: rgb(31,31,31);">.</span><span style="color: rgb(31,31,31);"> We anticipate further improvements through optimizations like I/O grouping (<a href="https://cwiki.apache.org/confluence/x/TYp3EQ" rel="nofollow">FLIP-426</a>) and leveraging the local disk as a secondary cache (FLIP-429).</span><span style="color: rgb(31,31,31);"> These advancements aim to bring HDFS performance closer to that of the local disk setup.</span></p></li></ul><p><span style="color: rgb(31,31,31);">The PoC running instructions are provided in the Appendix for those interested in exploring the setup further.</span></p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-RoadMap+LaunchingPlan"><span class="ne-text">Road Map + Launching Plan</span></h1><p><span>As discussed,</span><span> the disaggregated state evolution requires adjustments across the entire Flink Engine stack.</span><span> To ensure a smooth and successful transition,</span><span> we've divided the work into several stages with well-defined milestones:</span></p><p><strong>Milestone 1: Core Functionalities (Jun. 30th, 2024 - MVP Version):</strong></p><p><span>This phase focuses on building the foundational functionalities for disaggregated state management,</span><span> aiming to achieve an end-to-end baseline version by the end of the milestone.</span><span> This includes the following changes:</span></p><ul><li><strong>Asynchronous State APIs (FLIP-424):</strong><span> Introduce new APIs for asynchronous state access.</span></li><li><strong>Asynchronous Execution Model (FLIP-425):</strong><span> Implement a non-blocking execution model leveraging the asynchronous APIs introduced in FLIP-424.</span></li><li><strong>Grouping Remote State Access (FLIP-426):</strong><span> Enable retrieval of remote state data in batches.</span></li><li><strong>Disaggregated State Store (FLIP-427):</strong><span> Introduce the initial version of the ForSt disaggregated state store.</span></li><li><strong>Fault Tolerance/Rescale Integration (FLIP-428):</strong><span> Integrate checkpointing mechanisms with the disaggregated state store to ensure fault tolerance and rescaling capabilities.</span></li><li><p><strong>Optimized buffer checkpointing for Asynchronous Execution (FLIP-TBD): </strong>A more fine-grained buffer checkpointing strategy to optimize duration of sync phase. A set of declarative APIs will be introduced.</p></li></ul><p><span><strong>Milestone 2: Performance and U</strong></span><strong><span style="color: rgb(0,0,0);">sability Improvements (</span><span style="color: rgb(0,0,0);">Sept. 30th, 2024 – Flink 2.0 release, Beta Version):</span></strong></p><p><span>This phase prioritizes performance optimizations and user experience enhancements,</span><span> aiming to deliver a complete disaggregated state management system integrated with asynchronous SQL operators by the target date.</span><span> This milestone introduces several key functionalities:</span></p><ul><li><strong>Local Disk Caching (FLIP-429):</strong><span> Leverage local disks as a secondary cache to improve performance.</span></li><li><strong>TM State Ownership and Faster Checkpointing (FLIP-432):</strong><span> Enhance state management for Task Managers</span></li><li><strong>Disaggregated State Metrics (FLIP-431):</strong><span> Introduce disaggregated-state-related metrics</span><span>.</span></li><li><strong>SQL Operator Compatibility:</strong><span> Enable SQL async operator integration.</span></li></ul><p><span><strong>Milestone 3: Production Readiness and Stability (</strong></span><strong><span class="ne-text">Dec. 31st, 2024 – Flink 2.1 release, Prod-Ready):</span></strong></p><p><span>Milestone 3 prioritizes stability and production readiness,</span><span> aiming to deliver a reliable disaggregated state management system by the end of the year,</span><span> coinciding with the Flink 2.</span><span>1 release.</span><span> Key areas of focus during this phase include:</span></p><ul><li><strong>Stability Improvements</strong></li><li><strong>DataStream User Migration/Integration Tooling</strong></li><li><strong>Remote Compaction (FLIP-430)</strong></li></ul><p><br/></p><p>Notably, the disaggregated state backend can work with synchronous state access APIs (existing operators). However, t<span style="color: rgb(31,31,31);">his approach might lead to suboptimal performance due to blocking operations. To fully utilize the capability of disaggregated states, we strongly encourage users to use async state APIs.</span></p><p>SQL stateful operators will be reworked to provide async state access in Flink 2.0 to provide a complete end-to-end user experience.</p><p>DataStream users need to rewrite the stateful operators in a call-back way as demonstrated in the example. We will try to provide tools later to facilitate migration.</p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-RejectedAlternatives">Rejected Alternatives</h1><p>Tiered Storage: DFS as complimentary when the local disk is out of space. <span style="color: rgb(31,31,31);">While initially appealing due to its intuitive nature and potential performance benefits within local capacity,</span><span style="color: rgb(31,31,31);"> the tiered storage solution with local disk overflow to remote storage ultimately proves unsuitable for disaggregated state management in Flink.</span><span style="color: rgb(31,31,31);"> </span></p><p><span style="color: rgb(31,31,31);">Here's why:</span></p><ol><li><strong>Heavy Checkpointing Procedure:</strong><span><span> A considerable amount of files need to be uploaded during checkpointing.</span></span></li><li><strong>Limited Data Structure Flexibility:</strong><span> Confining local disk data to the SST format restricts potential performance gains from alternative caching structures.</span></li><li><strong>Inaccurate Warm/Cold Distinction:</strong><span> File-level classification of data as warm or cold inaccurately reflects actual access patterns,</span><span> leading to suboptimal resource allocation.</span></li><li><span><strong>More Complicated File Management:</strong> This architecture indicates that both local disk and DFS play part of the primary storage, hence </span><span>needs to unify the file management of the local disk and DFS, which is complicated in extreme cases of error handling e.t.c. </span></li></ol><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-References"><span>References</span></h1><p><span class="ne-text">[1] Disaggregated RocksDB: </span><span><a href="https://cwiki.apache.org/confluence/pages/createpage.action?spaceKey=FLINK&title=1&linkCreation=true&fromPageId=293046855" rel="nofollow">https://dl.acm.org/doi/pdf/10.1145/3589772</a></span></p><p><span><span class="ne-text" style="color: rgb(0,0,0);">[2] </span><span class="ne-text" style="color: rgb(0,0,0);">RocksDB-cloud/Rockset</span> <a class="external-link" href="https://github.com/rockset/rocksdb-cloud" rel="nofollow">https://github.com/rockset/rocksdb-cloud</a></span></p><h1 id="FLIP423:DisaggregatedStateStorageandManagement(UmbrellaFLIP)-Appendix:HowtorunthePoC"><span style="color: rgb(32,33,36);">Appendix: How to run the PoC</span></h1><p class="ne-p"><span class="ne-text">ForSt PoC Branch: </span><a class="external-link" href="https://github.com/ververica/ForSt/tree/disagg-poc" rel="nofollow"><span class="ne-text">https://github.com/ververica/ForSt/tree/disagg-poc</span></a></p><p class="ne-p"><span class="ne-text">Flink PoC Branch: </span><a class="external-link" href="https://github.com/ververica/flink-poc/tree/disagg-poc-2" rel="nofollow"><span class="ne-text">https://github.com/ververica/flink-poc/tree/disagg-poc-2</span></a></p><p class="ne-p"><span class="ne-text">Run PoC by following below three steps:</span></p><p class="ne-p"><strong><span class="ne-text">Step 1: Compile, Package and Install ForSt (Linux support ONLY)</span></strong></p><ul><li class="ne-p"><span class="ne-text">Download ForSt PoC branch.</span></li><li class="ne-p"><p><span class="ne-text">Compile and Package ForSt in the root directory, then a jar will be generated in "java/target"</span></p></li></ul><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: bash; gutter: false; theme: Default" data-theme="Default">DEBUG_LEVEL=0 ROCKSDB_DISABLE_JEMALLOC=true PORTABLE=1 CXXFLAGS="-Wno-error=shadow -Wno-error-defaulted-function-deleted -Wno-unknown-warning-option -Wno-error=unused-parameter -Wno-error=unused-variable" make -j8 rocksdbjavastatic</pre>
</div></div><ul><li>Install ForSt jar to local</li></ul><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: java; gutter: false; theme: Default" data-theme="Default">mvn install:install-file -Dfile=/path/to/your/library.jar -DgroupId=com.ververica -DartifactId=forstjni -Dversion=8.5.3-poc-SNAPSHOT -Dpackaging=jar</pre>
</div></div><p><strong>Step 2: </strong><strong><span class="ne-text">Package Flink and WordCount Jar</span></strong></p><ul><li><span class="ne-text">Download Flink PoC branch</span></li><li><span class="ne-text">Package Flink and streaming job jar.</span></li></ul><div class="code panel pdl" style="border-width: 1px;"><div class="codeContent panelContent pdl">
<pre class="syntaxhighlighter-pre" data-syntaxhighlighter-params="brush: bash; gutter: false; theme: Default" data-theme="Default">mvn clean package -DskipTests -Dcheckstyle.skip=true -Dspotless.check.skip=true -Denforcer.skip=true -Drat.skip=true -Djapicmp.skip=true</pre>
</div></div><ul><li>The Flink binary can be found at "flink-dist/target/flink-1.19-SNAPSHOT-bin/flink-1.19-SNAPSHOT" and the streaming test jar exists in "flink-examples/flink-examples-streaming/flink-examples-streaming-1.19-SNAPSHOT-AsyncWordCount.jar".</li></ul><p><strong>Step 3: Run WordCount</strong></p><ul><li>Use Flink CLI command to submit a job by a jar. Note: Place the right dependency jar in plugin/ directory for remote file system.</li></ul><p><br/></p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=293046855">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=293046855&revisedVersion=102&originalVersion=101">View Changes Online</a>
</div>
</div>Zakelly Lan2024-02-23T02:47:28ZFLIP-425: Asynchronous Execution Model > drain.pngZakelly Lantag:cwiki.apache.org,2009:attachment-298781339-12024-03-27T09:05:17Z2024-03-27T09:05:17Z<div class="feed"> <p>File
<b>attached</b> by
<a href=" https://cwiki.apache.org/confluence/display/~zakelly
">Zakelly Lan</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<p>
<span class="aui-icon content-type-attachment-image" title="PNG File">PNG File</span> <a href="https://cwiki.apache.org/confluence/download/attachments/293046859/drain.png?version=1&modificationDate=1711530317000&api=v2">drain.png</a> <span class="smalltext">(42 kB)</span>
<br/>
</p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/pages/viewpageattachments.action?pageId=293046859">View Attachments</a>
</div>
</div>Zakelly Lan2024-03-27T09:05:17ZFlink Release ManagementMatthias Pohltag:cwiki.apache.org,2009:page-65145746-272024-03-27T07:58:00Z2016-07-11T10:49:17Z<div class="feed"> <p>
Page
<b>edited</b> by
<a href=" https://cwiki.apache.org/confluence/display/~mapohl
">Matthias Pohl</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<div class="confluence-information-macro confluence-information-macro-information"><span class="aui-icon aui-icon-small aui-iconfont-info confluence-information-macro-icon"></span><div class="confluence-information-macro-body"><p>This page is work in progress! </p></div></div><p>Flink Release Management consists of the following tasks:</p><ul><li>Managing the feature development for the release</li><li>Maintenance of CI builds and infrastructure (for <code>master</code> and the two most-recently release Flink versions)</li><li>Jira maintenance</li></ul><p>These tasks are not necessarily exclusively done by the release manager. The community should take care of it. The responsibility of the release manager is to make sure that a certain quality and stability of the relevant code base is achieved.</p><p>Additionally, any release-related documentation should be kept up-to-date (e.g. <a href="https://cwiki.apache.org/confluence/display/FLINK/Release+Management+and+Feature+Plan">Release Management and Feature Plan</a> or <a href="https://cwiki.apache.org/confluence/display/FLINK/Creating+a+Flink+Release">Creating a Flink Release</a>)</p><h2 id="FlinkReleaseManagement-Organization">Organization</h2><h3 id="FlinkReleaseManagement-PreparingaReleaseCycle">Preparing a Release Cycle</h3><ul><li>Setting up a release page (see <a href="https://cwiki.apache.org/confluence/display/FLINK/1.17+Release">1.17 Release</a> as a template)</li><li>Announcing the plan for the release cycle (e.g. feature freeze date)</li></ul><h3 id="FlinkReleaseManagement-RegularSync">Regular Sync</h3><p>It's a good habit to meet on a regular basis to sync on the developments of the current release cycle. A summary should be kept in the release's wiki article (see <a href="https://cwiki.apache.org/confluence/display/FLINK/Release+Management+and+Feature+Plan">Release Management and Feature Plan</a>) and sent to the dev mailing list to keep the community up-to-date.</p><h3 id="FlinkReleaseManagement-FeatureFreeze">Feature Freeze</h3><p>The feature freeze is a set date until which features can be added to <code>master</code>. After the feature freeze, no additional feature are allowed to be merged into <code>master</code>. Only bugfixes and documentation changes are allowed. The goal is to stabilize <code>master</code> before cutting of the release branch. The feature freeze date is communicated at the beginning of a release cycle. It's not uncommon that the date will be changed during the release cycle if there are valid reasons to do so. Such a decision needs to be discussed in the dev mailing list (see <a class="external-link" href="https://lists.apache.org/thread/6v9mjx1jgf4l083hqw54rp4cf5fsnfom" rel="nofollow">1.17 feature freeze extension discussion</a>).</p><p>The time between announcing the feature freeze and cutting the release branch should be as short as possible since it's blocking work that should go into future releases.</p><h3 id="FlinkReleaseManagement-ReleaseTesting">Release Testing</h3><p>Release testing happens after the release branch is cut and CI is stable enough. The goal is to test features manually that ended up in this build. Additionally, the documentation for these feature should be available. Any blocking issues that come up during the release testing need to be addressed before going forward with the release.</p><h3 id="FlinkReleaseManagement-ReleaseMetrics">Release Metrics </h3><ul><li>Count contributors, the following git commands can be used to count contributors for given commit range of current branch:<ul><li><code>git shortlog --summary startCommitId..endCommitId | awk -F ' ' '{$1=""; print $0 }'|sort -n|awk 'BEGIN{ORS=", "}{print $0}'</code> </li></ul></li></ul><ul><li>Count resolved issues, the JIRA filter can be used to count the resolved issues in this version:<ul><li><code>project = flink AND status in (closed, resolved, Fixed, Completed, Done) AND fixVersion in (1.17.0)</code> </li></ul></li></ul><h2 id="FlinkReleaseManagement-MaintenanceofCIbuildsandinfrastructure">Maintenance of CI builds and infrastructure</h2><p>The release manager should ensure stability of <code>master</code> and the two most-recently published Flink versions in terms of CI. Builds can be monitored on <a class="external-link" href="https://dev.azure.com/apache-flink/apache-flink/_build?definitionId=1&_a=summary" rel="nofollow">AzureCI Flink build overview</a> (see <a href="https://cwiki.apache.org/confluence/display/FLINK/Testing+Infrastructure">Testing Infrastructure</a> for further details on the build process).</p><h3 id="FlinkReleaseManagement-MonitoringCIfailuresinmasterandthereleasebranches">Monitoring CI failures in master and the release branches</h3><p>Failed builds are reported to <a class="external-link" href="https://apache-flink.slack.com/archives/C03MR1HQHK2" rel="nofollow">Apache Flink's #builds Slack channel</a>. Build failures should be investigated and documented in this Slack channel (i.e. linking the corresponding ticket in the Slack thread and marking the thread with a "check" emoj when the investigation is done for this build). The documentation in the Slack channel allows us to work concurrently on CI failures (i.e. a missing check mark for a build failure means that the build was not fully investigated, yet, and should be picked up).</p><p>There is an issue with clicking Azure Pipeline links that are reported in the Slack channel. You need to install a redirect routine in your browser to make this work. The instructions can be found in the <a class="external-link" href="https://apache-flink.slack.com/canvas/C03MR1HQHK2" rel="nofollow">#builds channels canvas</a>.</p><p>Other tasks:</p><ul><li>Monitoring the remote branches. Sometimes, there are remote branches created accidentally in the Apache Flink repo. Branches should generally been created in the forks. We might want to reach out to contributors to delete these accidentally created remote branches. The following branches shouldn't be touched:<br/><ul><li><code>master</code> & <code>release-*</code> - Flink versioning branches</li><li><code>blink</code> - Branch holding the legacy blink code. This one is kept for historical purposes.</li><li><code>experiment_gha_docs</code> & <code>exp_github_actions</code> - These branches are kept as part of the Github Actions migration efforts (see <a class="confluence-userlink user-mention" data-username="chesnay" href="https://cwiki.apache.org/confluence/display/~chesnay" data-linked-resource-id="61330986" data-linked-resource-version="1" data-linked-resource-type="userinfo" data-base-url="https://cwiki.apache.org/confluence">Chesnay Schepler</a>'s comment in the <a class="external-link" href="https://lists.apache.org/thread/cvqh07m7b45mzk0pyj9t1yv9m04wkrff" rel="nofollow">related ML post</a>).</li><li><code>dependabot/*</code> - These branches are temporarily created by dependabot for version bumps (related <a class="external-link" href="https://lists.apache.org/thread/c2ftht7227t2tobrtjmrjyjx9kpmgm70" rel="nofollow">ML announcement</a>).</li></ul></li></ul><h3 id="FlinkReleaseManagement-RelevantRepositories,Workflowsandotherartifacts">Relevant Repositories, Workflows and other artifacts</h3><ul><li><a class="external-link" href="https://github.com/apache/flink" rel="nofollow">apache/flink</a><ul><li>Main repository for Apache Flink</li><li>Azure Pipelines (YAML configuration: <a class="external-link" href="https://github.com/apache/flink/blob/master/tools/azure-pipelines/build-apache-repo.yml" rel="nofollow">apache/flink:tools/azure-pipelines/build-apache-repo.yml</a>)<br/><ul><li><a class="external-link" href="https://dev.azure.com/apache-flink/apache-flink/_build?definitionId=1&_a=summary" rel="nofollow">Nightly builds</a> </li><li><a class="external-link" href="https://dev.azure.com/apache-flink/apache-flink/_build?definitionId=2" rel="nofollow">PR builds</a></li><li>Azure Pipelines can be used for forks as well (but require a special setup; see <a href="https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-(deprecated)Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository" rel="nofollow">wiki</a>). The YAML configuration for this CI is <a class="external-link" href="https://github.com/apache/flink/blob/master/azure-pipelines.yml" rel="nofollow">apache/flink:azure-pipelines</a>. This is approach is deprecated because the free Azure Pipelines offer for OpenSource projects is harder to get. Use the GitHub Actions workflow, instead.</li></ul></li><li>GitHub Actions (YAML configuration under <a class="external-link" href="https://github.com/apache/flink/tree/master/.github" rel="nofollow">apache/flink:.github</a>)<ul><li>Currently in beta (<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-396%3A+Trial+to+test+GitHub+Actions+as+an+alternative+for+Flink%27s+current+Azure+CI+infrastructure" rel="nofollow">FLIP-396</a>)</li><li>Support added for 1.18+ (i.e. Azure Pipelines can be only deprecated after deprecating 1.17)</li><li>
<span class="jira-issue" data-jira-key="FLINK-33901" >
<a href="https://issues.apache.org/jira/browse/FLINK-33901" class="jira-issue-key"><img class="icon"
src="https://issues.apache.org/jira/secure/viewavatar?size=xsmall&avatarId=21141&avatarType=issuetype"/>FLINK-33901</a>
-
<span class="summary">Trial Period: GitHub Actions</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-default
jira-macro-single-issue-export-pdf">Open</span>
</span>
collects the subtasks to finalize the GHA migration.</li><li><a class="external-link" href="https://github.com/apache/flink/actions/workflows/nightly.yml" rel="nofollow">Nightly builds</a> (see FLINK-33901 for missing pieces like nightly artifact upload and Slack integration)</li><li><a class="external-link" href="https://github.com/apache/flink/actions/workflows/ci.yml" rel="nofollow">Pushes to master or the release branches</a> (the same workflow also runs on forks for pushes)</li><li><a class="external-link" href="https://github.com/apache/flink/actions/workflows/docs.yml" rel="nofollow">Nightly docs builds</a> (could be merged into nightly builds;
<span class="jira-issue" data-jira-key="FLINK-34045" >
<a href="https://issues.apache.org/jira/browse/FLINK-34045" class="jira-issue-key"><img class="icon"
src="https://issues.apache.org/jira/secure/viewavatar?size=xsmall&avatarId=21146&avatarType=issuetype"/>FLINK-34045</a>
-
<span class="summary">Add docs generation to workflow</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-current
jira-macro-single-issue-export-pdf">In Progress</span>
</span>
)</li></ul></li></ul></li><li><a class="external-link" href="https://github.com/apache/flink-shaded/" rel="nofollow">apache/flink-shaded</a><ul><li>Used for shaded dependencies of Apache Flink</li><li>Separate release cycle (), i.e.<ol><li>Do version bumps in <code>apache/flink-shaded</code> </li><li><a href="https://cwiki.apache.org/confluence/display/FLINK/Creating+a+flink-shaded+release" rel="nofollow">Release apache/flink-shaded</a></li><li>Upgrade flink-shaded dependencies in <code>apache/flink</code> </li></ol></li><li>Especially for the netty dependencies it is adviced to do version upgrades early on in a Flink release cycle to allow for more CI runs to verify the change</li></ul></li><li><a class="external-link" href="https://github.com/apache/flink-docker/" rel="nofollow">apache/flink-docker</a><ul><li>Flink Docker images that are pushed to the <a class="external-link" href="https://hub.docker.com/_/flink" rel="nofollow">Apache Docker Hub registry</a> for each release</li><li><a class="external-link" href="https://github.com/apache/flink-docker/actions/workflows/snapshot.yml" rel="nofollow">Nightly versions</a> for <code>master</code> and the <code>release-*</code> branches are pushed to GitHub Container Registry and used in Flink's nightly builds</li></ul></li><li><a class="external-link" href="https://github.com/apache/flink-web" rel="nofollow">apache/flink-web</a><ul><li>Repository for <a class="external-link" href="https://flink.apache.org/" rel="nofollow">flink.apache.org</a></li><li><code>asf-site</code> is the repo's main branch (deployment happens automatically through Apache Infra)</li><li>Website needs to be manually build with every change (should be done through GitHub Actions similarly to what is done for the Flink docs in <a class="external-link" href="https://github.com/apache/flink/actions/workflows/docs.yml" rel="nofollow">apache/flink:.github/workflows/docs.yml</a>)</li></ul></li><li><a class="external-link" href="https://github.com/apache/flink-connector-shared-utils" rel="nofollow">apache/flink-connector-shared-utils</a><ul><li>Common code for all Flink connectors</li><li>Organized in branches (see <a class="external-link" href="https://github.com/apache/flink-connector-shared-utils/blob/main/README.md" rel="nofollow">apache/flink-connector-shared-utils:README.md</a> for further details on the structure)</li></ul></li><li><code>apache/flink-connector*</code> , e.g. <a class="external-link" href="https://github.com/apache/flink-connector-kafka" rel="nofollow">apache/flink-connector-kafka</a>, <a class="external-link" href="https://github.com/apache/flink-connector-jdbc" rel="nofollow">apache/flink-connector-jdbc</a>, ...</li><li><a class="external-link" href="https://github.com/flink-ci/" rel="nofollow">flink-ci</a> Github organization:<ul><li>Ververica-owned organization with CI-related code</li><li>See <a href="https://cwiki.apache.org/confluence/display/FLINK/Continuous+Integration#ContinuousIntegration-Repositories">Continuous Integration#Repositories</a> for further details on the different repos</li><li><a class="external-link" href="https://github.com/flink-ci/flink-ci-docker" rel="nofollow">flink-ci/flink-ci-docker</a> (<strong>The process below is not ideal and should change to use some apache-owned repo;
<span class="jira-issue" data-jira-key="FLINK-34695" >
<a href="https://issues.apache.org/jira/browse/FLINK-34695" class="jira-issue-key"><img class="icon"
src="https://issues.apache.org/jira/secure/viewavatar?size=xsmall&avatarId=21140&avatarType=issuetype"/>FLINK-34695</a>
-
<span class="summary">Move Flink's CI docker container into a public repo</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-default
jira-macro-single-issue-export-pdf">Open</span>
</span>
</strong>)<ul><li>Docker container that is used in different CI workflows (Azure Pipelines and GitHub Actions)</li><li>Repository is not really used right now (due to ownership issues, slow response)</li><li>Instead, changes were pushed to <a class="external-link" href="https://github.com/zentol/flink-ci-docker" rel="nofollow">zentol/flink-ci-docker</a></li><li>The docker containers where pushed to "private" Docker (e.g. <a class="external-link" href="https://hub.docker.com/r/rmetzger/flink-ci/tags" rel="nofollow">rmetzger/flink-ci</a>, <a class="external-link" href="https://hub.docker.com/r/chesnay/flink-ci/tags" rel="nofollow">chesnay/flink-ci</a>, <a class="external-link" href="https://hub.docker.com/r/mapohl/flink-ci" rel="nofollow">mapohl/flink-ci</a>) registries in the past.</li></ul></li></ul></li></ul><h2 id="FlinkReleaseManagement-PerformanceRegressionTests">Performance Regression Tests</h2><p>Performance regression tests are used to monitor that there are no changes that reduce the performance of Flink. There is more documentation on this topic in <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=115511847">Codespeed / Benchmarks</a>. Regressions are reported in <a class="external-link" href="https://apache-flink.slack.com/archives/C0471S0DFJ9" rel="nofollow">Apache Flink's #flink-dev-benchmarks Slack channel</a>.</p><h2 id="FlinkReleaseManagement-Jiramaintenance">Jira maintenance</h2><ul><li>Build failures should be reported in the corresponding Jira issue (or a Jira issue should be created if none exists, yet). Contributors should be pinged to fix instabilities as soon as possible to ensure a stable infrastructure of the course of the release cycle. More details on Jira issues can be found on the <a href="https://cwiki.apache.org/confluence/display/FLINK/Flink+Jira+Process">Flink Jira Process</a> wiki page.</li><li>Newly created Jira issue should follow the <a href="https://cwiki.apache.org/confluence/display/FLINK/Flink+Jira+Process" rel="nofollow">Flink Jira Process guide</a> (e.g. <code>fixVersion</code> , <code>affectedVersion</code> , <code>component</code> and have the label <code>test-stability</code>)</li><li>Important information to improve Jira issue search:<ul><li>Name of the test that failed</li><li>Link to the test failure (ideally with the relevant log line; both Azure Pipelines and GitHub Actions support log line-specific links)</li><li>Log snippet that identified the test failure (e.g. assertion error or stacktrace)</li></ul></li></ul><h3 id="FlinkReleaseManagement-HintsaroundAzureCI/Jirausage">Hints around AzureCI/Jira usage</h3><ul><li>Console log output can be linked per log line:<ul><li>GitHub Actions: Click the line number on the left side of the console view to generate the line-specific URL.</li><li>Azure Pipelines: The link button will appear at the end (i.e. the right side) of the log line when hovering over the log line.</li></ul></li><li>There are several URLs with placeholder (i.e. <code>%s</code>) that might be handy when accessing Jira through your browser using <a class="external-link" href="https://www-archive.mozilla.org/docs/end-user/keywords.html" rel="nofollow">Firefox's bookmark keywords</a> or <a class="external-link" href="https://support.google.com/chrome/answer/95426?hl=en&co=GENIE.Platform%3DDesktop" rel="nofollow">Chrome's search engine feature</a>:<ul><li>Jira issue look up by ID (e.g. "<keyword> 123 would lead to
<span class="jira-issue resolved" data-jira-key="FLINK-123" >
<a href="https://issues.apache.org/jira/browse/FLINK-123" class="jira-issue-key"><img class="icon"
src="https://issues.apache.org/jira/secure/viewavatar?size=xsmall&avatarId=21133&avatarType=issuetype"/>FLINK-123</a>
-
<span class="summary">Strange (and faulty) Iterations Behaviour</span>
<span class="aui-lozenge aui-lozenge-subtle aui-lozenge-success
jira-macro-single-issue-export-pdf">Closed</span>
</span>
): <br/><code><span class="nolink">https://issues.apache.org/jira/browse/FLINK-%s</span></code></li><li>Search for open or closed Jira issues with a substring (this is handy to find test stability issues): <br/><code><span class="nolink"><span class="nolink">https://issues.apache.org/jira/browse/FLINK-0?filter=-1&jql=project%20%3D%20FLINK%20AND%20text%20~%20%22%s%22%20ORDER%20BY%20id%20DESC</span></span></code></li><li>Same as above only for opened issues: <br/><code><span class="nolink">https://issues.apache.org/jira/browse/FLINK-0?filter=-1&jql=project%20%3D%20FLINK%20AND%20text%20~%20%22%s%22%20AND%20status%20NOT%20IN%20(Closed%2C%20Resolved)</span></code></li><li><span style="letter-spacing: 0.0px;">Look for most-recently updated test-stability Jira issues by date (number reflects the date range since today): <br/><code><span class="nolink">https://issues.apache.org/jira/browse/FLINK-0?jql=project%20%3D%20FLINK%20AND%20issuetype%20%3D%20Bug%20AND%20labels%20%3D%20test-stability%20%20AND%20status%20NOT%20IN%20(Closed%2C%20Resolved)%20%20AND%20updatedDate%20%3E%20startOfDay(-%sd)%20ORDER%20BY%20updatedDate%20DESC</span></code></span></li></ul></li></ul>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/display/FLINK/Flink+Release+Management">View Online</a>
·
<a href="https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=65145746&revisedVersion=27&originalVersion=26">View Changes Online</a>
</div>
</div>Matthias Pohl2016-07-11T10:49:17ZFLIP-441: Show the JobType and remove Execution Mode on Flink WebUI > image-2024-3-27_14-12-2.pngRui Fantag:cwiki.apache.org,2009:attachment-298781323-12024-03-27T06:12:03Z2024-03-27T06:12:03Z<div class="feed"> <p>File
<b>attached</b> by
<a href=" https://cwiki.apache.org/confluence/display/~fanrui
">Rui Fan</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<p>
<span class="aui-icon content-type-attachment-image" title="PNG File">PNG File</span> <a href="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-27_14-12-2.png?version=1&modificationDate=1711519923000&api=v2">image-2024-3-27_14-12-2.png</a> <span class="smalltext">(19 kB)</span>
<br/>
</p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/pages/viewpageattachments.action?pageId=298781290">View Attachments</a>
</div>
</div>Rui Fan2024-03-27T06:12:03ZFLIP-441: Show the JobType and remove Execution Mode on Flink WebUI > image-2024-3-27_13-46-54.pngRui Fantag:cwiki.apache.org,2009:attachment-298781321-12024-03-27T05:46:56Z2024-03-27T05:46:56Z<div class="feed"> <p>File
<b>attached</b> by
<a href=" https://cwiki.apache.org/confluence/display/~fanrui
">Rui Fan</a>
</p>
<div style="border-top: 1px solid #ddd; border-bottom: 1px solid #ddd; padding: 10px;">
<p>
<span class="aui-icon content-type-attachment-image" title="PNG File">PNG File</span> <a href="https://cwiki.apache.org/confluence/download/attachments/298781290/image-2024-3-27_13-46-54.png?version=1&modificationDate=1711518416000&api=v2">image-2024-3-27_13-46-54.png</a> <span class="smalltext">(29 kB)</span>
<br/>
</p>
</div>
<div style="padding: 10px 0;">
<a href="https://cwiki.apache.org/confluence/pages/viewpageattachments.action?pageId=298781290">View Attachments</a>
</div>
</div>Rui Fan2024-03-27T05:46:56Z