diff --git a/paimon-benchmark/paimon-cluster-benchmark/src/main/java/org/apache/paimon/benchmark/QueryRunner.java b/paimon-benchmark/paimon-cluster-benchmark/src/main/java/org/apache/paimon/benchmark/QueryRunner.java
index b07cdef8465e..8bfe4b6c9c03 100644
--- a/paimon-benchmark/paimon-cluster-benchmark/src/main/java/org/apache/paimon/benchmark/QueryRunner.java
+++ b/paimon-benchmark/paimon-cluster-benchmark/src/main/java/org/apache/paimon/benchmark/QueryRunner.java
@@ -77,7 +77,7 @@ public Result run() {
String sinkPathConfig =
BenchmarkGlobalConfiguration.loadConfiguration()
- .getString(BenchmarkOptions.SINK_PATH);
+ .get(BenchmarkOptions.SINK_PATH);
if (sinkPathConfig == null) {
throw new IllegalArgumentException(
BenchmarkOptions.SINK_PATH.key() + " must be set");
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/JobInfo.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/JobInfo.java
new file mode 100644
index 000000000000..38e3bfea6965
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/JobInfo.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** The {@link JobInfo} represents the meta information of current job. */
+@PublicEvolving
+public interface JobInfo {
+
+ /**
+ * Get the ID of the job.
+ *
+ * @return the ID of the job
+ */
+ JobID getJobId();
+
+ /**
+ * Get the name of the job.
+ *
+ * @return the name of the job
+ */
+ String getJobName();
+}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/SupportsConcurrentExecutionAttempts.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/SupportsConcurrentExecutionAttempts.java
new file mode 100644
index 000000000000..a8a61d72883a
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/SupportsConcurrentExecutionAttempts.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * The interface indicates that it supports multiple attempts executing at the same time.
+ *
+ *
Currently, the interface is used for speculative execution. If a sink implementation (SinkV2,
+ * OutputFormat or SinkFunction) inherits this interface, the sink operator would be considered to
+ * support speculative execution.
+ */
+@PublicEvolving
+public interface SupportsConcurrentExecutionAttempts {}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java
new file mode 100644
index 000000000000..21fca4e4c319
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** The default implementation of {@link OpenContext}. */
+@PublicEvolving
+public class DefaultOpenContext implements OpenContext {
+
+ public static final OpenContext INSTANCE = new DefaultOpenContext();
+}
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
similarity index 65%
rename from paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java
rename to paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
index 6eabd1db7f38..4ff5484b3b08 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
@@ -16,19 +16,14 @@
* limitations under the License.
*/
-package org.apache.flink.table.catalog;
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
/**
- * Dummy placeholder to resolve compatibility issue of CatalogMaterializedTable(introduced in flink
- * 1.20).
+ * The {@link OpenContext} interface provides necessary information required by the {@link
+ * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be
+ * used to add more methods without affecting the signature of {@code RichFunction#open}.
*/
-public interface CatalogMaterializedTable extends CatalogBaseTable {
- /** Dummy LogicalRefreshMode placeholder. */
- enum LogicalRefreshMode {}
-
- /** Dummy RefreshMode placeholder. */
- enum RefreshMode {}
-
- /** Dummy RefreshStatus placeholder. */
- enum RefreshStatus {}
-}
+@PublicEvolving
+public interface OpenContext {}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/RichFunction.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
new file mode 100644
index 000000000000..ae83fb30f2bd
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * An base interface for all rich user-defined functions. This class defines methods for the life
+ * cycle of the functions, as well as methods to access the context in which the functions are
+ * executed.
+ */
+@Public
+public interface RichFunction extends Function {
+
+ /**
+ * Initialization method for the function. It is called before the actual working methods (like
+ * map or join) and thus suitable for one time setup work. For functions that are
+ * part of an iteration, this method will be invoked at the beginning of each iteration
+ * superstep.
+ *
+ *
The configuration object passed to the function can be used for configuration and
+ * initialization. The configuration contains all parameters that were configured on the
+ * function in the program composition.
+ *
+ *
{@code
+ * public class MyFilter extends RichFilterFunction {
+ *
+ * private String searchString;
+ *
+ * public void open(Configuration parameters) {
+ * this.searchString = parameters.getString("foo");
+ * }
+ *
+ * public boolean filter(String value) {
+ * return value.equals(searchString);
+ * }
+ * }
+ * }
+ *
+ * By default, this method does nothing.
+ *
+ * @param parameters The configuration containing the parameters attached to the contract.
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ * @see org.apache.flink.configuration.Configuration
+ * @deprecated This method is deprecated since Flink 1.19. The users are recommended to
+ * implement {@code open(OpenContext openContext)} and implement {@code open(Configuration
+ * parameters)} with an empty body instead. 1. If you implement {@code open(OpenContext
+ * openContext)}, the {@code open(OpenContext openContext)} will be invoked and the {@code
+ * open(Configuration parameters)} won't be invoked. 2. If you don't implement {@code
+ * open(OpenContext openContext)}, the {@code open(Configuration parameters)} will be
+ * invoked in the default implementation of the {@code open(OpenContext openContext)}.
+ * @see
+ * FLIP-344: Remove parameter in RichFunction#open
+ */
+ @Deprecated
+ void open(Configuration parameters) throws Exception;
+
+ /**
+ * Initialization method for the function. It is called before the actual working methods (like
+ * map or join) and thus suitable for one time setup work. For functions that are
+ * part of an iteration, this method will be invoked at the beginning of each iteration
+ * superstep.
+ *
+ *
The openContext object passed to the function can be used for configuration and
+ * initialization. The openContext contains some necessary information that were configured on
+ * the function in the program composition.
+ *
+ *
{@code
+ * public class MyFilter extends RichFilterFunction {
+ *
+ * private String searchString;
+ *
+ * public void open(OpenContext openContext) {
+ * // initialize the value of searchString
+ * }
+ *
+ * public boolean filter(String value) {
+ * return value.equals(searchString);
+ * }
+ * }
+ * }
+ *
+ * By default, this method does nothing.
+ *
+ *
1. If you implement {@code open(OpenContext openContext)}, the {@code open(OpenContext
+ * openContext)} will be invoked and the {@code open(Configuration parameters)} won't be
+ * invoked. 2. If you don't implement {@code open(OpenContext openContext)}, the {@code
+ * open(Configuration parameters)} will be invoked in the default implementation of the {@code
+ * open(OpenContext openContext)}.
+ *
+ * @param openContext The context containing information about the context in which the function
+ * is opened.
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ */
+ @PublicEvolving
+ default void open(OpenContext openContext) throws Exception {
+ open(new Configuration());
+ }
+
+ /**
+ * Tear-down method for the user code. It is called after the last call to the main working
+ * methods (e.g. map or join). For functions that are part of an iteration, this
+ * method will be invoked after each iteration superstep.
+ *
+ *
This method can be used for clean up work.
+ *
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ */
+ void close() throws Exception;
+
+ // ------------------------------------------------------------------------
+ // Runtime context
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the context that contains information about the UDF's runtime, such as the parallelism
+ * of the function, the subtask index of the function, or the name of the task that executes the
+ * function.
+ *
+ *
The RuntimeContext also gives access to the {@link
+ * org.apache.flink.api.common.accumulators.Accumulator}s and the {@link
+ * org.apache.flink.api.common.cache.DistributedCache}.
+ *
+ * @return The UDF's runtime context.
+ */
+ RuntimeContext getRuntimeContext();
+
+ /**
+ * Gets a specialized version of the {@link RuntimeContext}, which has additional information
+ * about the iteration in which the function is executed. This IterationRuntimeContext is only
+ * available if the function is part of an iteration. Otherwise, this method throws an
+ * exception.
+ *
+ * @return The IterationRuntimeContext.
+ * @throws java.lang.IllegalStateException Thrown, if the function is not executed as part of an
+ * iteration.
+ */
+ IterationRuntimeContext getIterationRuntimeContext();
+
+ /**
+ * Sets the function's runtime context. Called by the framework when creating a parallel
+ * instance of the function.
+ *
+ * @param t The runtime context.
+ */
+ void setRuntimeContext(RuntimeContext t);
+}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java
new file mode 100644
index 000000000000..5e13a9b62329
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.serialization;
+
+import com.esotericsoftware.kryo.Serializer;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInfoFactory;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.util.TernaryBoolean;
+
+import java.io.Serializable;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+
+/**
+ * A config to define the behavior for serializers in Flink job, it manages the registered types and
+ * serializers. The config is created from job configuration and used by Flink to create serializers
+ * for data types.
+ */
+@PublicEvolving
+public interface SerializerConfig extends Serializable {
+ /**
+ * Adds a new Kryo default serializer to the Runtime.
+ *
+ *
Note that the serializer instance must be serializable (as defined by
+ * java.io.Serializable), because it may be distributed to the worker nodes by java
+ * serialization.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializer The serializer to use.
+ */
+ @Internal
+ & Serializable> void addDefaultKryoSerializer(
+ Class> type, T serializer);
+
+ /**
+ * Adds a new Kryo default serializer to the Runtime.
+ *
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializerClass The class of the serializer to use.
+ */
+ @Internal
+ void addDefaultKryoSerializer(Class> type, Class extends Serializer>> serializerClass);
+
+ /**
+ * Registers the given type with a Kryo Serializer.
+ *
+ *
Note that the serializer instance must be serializable (as defined by
+ * java.io.Serializable), because it may be distributed to the worker nodes by java
+ * serialization.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializer The serializer to use.
+ */
+ @Internal
+ & Serializable> void registerTypeWithKryoSerializer(
+ Class> type, T serializer);
+
+ /**
+ * Registers the given Serializer via its class as a serializer for the given type at the
+ * KryoSerializer.
+ *
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializerClass The class of the serializer to use.
+ */
+ @Internal
+ @SuppressWarnings("rawtypes")
+ void registerTypeWithKryoSerializer(Class> type, Class extends Serializer> serializerClass);
+
+ /**
+ * Registers the given type with the serialization stack. If the type is eventually serialized
+ * as a POJO, then the type is registered with the POJO serializer. If the type ends up being
+ * serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
+ * written.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the type to register.
+ */
+ @Internal
+ void registerPojoType(Class> type);
+
+ /**
+ * Registers the given type with the serialization stack. If the type is eventually serialized
+ * as a POJO, then the type is registered with the POJO serializer. If the type ends up being
+ * serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
+ * written.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the type to register.
+ */
+ @Internal
+ void registerKryoType(Class> type);
+
+ /**
+ * Returns the registered types with Kryo Serializers.
+ *
+ * @deprecated The method is deprecated because instance-type Kryo serializer definition based
+ * on {@link ExecutionConfig.SerializableSerializer} is deprecated. Use class-type Kryo
+ * serializers instead.
+ */
+ @Deprecated
+ LinkedHashMap, ExecutionConfig.SerializableSerializer>>
+ getRegisteredTypesWithKryoSerializers();
+
+ /** Returns the registered types with their Kryo Serializer classes. */
+ LinkedHashMap, Class extends Serializer>>>
+ getRegisteredTypesWithKryoSerializerClasses();
+
+ /**
+ * Returns the registered default Kryo Serializers.
+ *
+ * @deprecated The method is deprecated because {@link ExecutionConfig.SerializableSerializer}
+ * is deprecated.
+ */
+ @Deprecated
+ LinkedHashMap, ExecutionConfig.SerializableSerializer>> getDefaultKryoSerializers();
+
+ /** Returns the registered default Kryo Serializer classes. */
+ LinkedHashMap, Class extends Serializer>>> getDefaultKryoSerializerClasses();
+
+ /** Returns the registered Kryo types. */
+ LinkedHashSet> getRegisteredKryoTypes();
+
+ /** Returns the registered POJO types. */
+ LinkedHashSet> getRegisteredPojoTypes();
+
+ /** Returns the registered type info factories. */
+ Map, Class extends TypeInfoFactory>>> getRegisteredTypeInfoFactories();
+
+ /**
+ * Checks whether generic types are supported. Generic types are types that go through Kryo
+ * during serialization.
+ *
+ * Generic types are enabled by default.
+ */
+ boolean hasGenericTypesDisabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setGenericTypes(boolean genericTypes);
+
+ /** Returns whether Kryo is the serializer for POJOs. */
+ boolean isForceKryoEnabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setForceKryo(boolean forceKryo);
+
+ /** Returns whether the Apache Avro is the serializer for POJOs. */
+ boolean isForceAvroEnabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setForceAvro(boolean forceAvro);
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ public void setForceKryoAvro(boolean forceKryoAvro);
+
+ /** Returns whether forces Flink to register Apache Avro classes in Kryo serializer. */
+ TernaryBoolean isForceKryoAvroEnabled();
+
+ /**
+ * Sets all relevant options contained in the {@link ReadableConfig} such as e.g. {@link
+ * PipelineOptions#FORCE_KRYO}.
+ *
+ *
It will change the value of a setting only if a corresponding option was set in the {@code
+ * configuration}. If a key is not present, the current value of a field will remain untouched.
+ *
+ * @param configuration a configuration to read the values from
+ * @param classLoader a class loader to use when loading classes
+ */
+ void configure(ReadableConfig configuration, ClassLoader classLoader);
+
+ SerializerConfig copy();
+}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java
new file mode 100644
index 000000000000..ae637d6456e7
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.connector.sink2;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobInfo;
+import org.apache.flink.api.common.TaskInfo;
+
+import java.util.OptionalLong;
+
+/**
+ * Common interface which exposes runtime info for creating {@link SinkWriter} and {@link Committer}
+ * objects.
+ */
+@Internal
+public interface InitContext {
+ /**
+ * The first checkpoint id when an application is started and not recovered from a previously
+ * taken checkpoint or savepoint.
+ */
+ long INITIAL_CHECKPOINT_ID = 1;
+
+ /**
+ * Get the id of task where the committer is running.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getSubtaskId() {
+ return getTaskInfo().getIndexOfThisSubtask();
+ }
+
+ /**
+ * Get the number of parallel committer tasks.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getNumberOfParallelSubtasks() {
+ return getTaskInfo().getNumberOfParallelSubtasks();
+ }
+
+ /**
+ * Gets the attempt number of this parallel subtask. First attempt is numbered 0.
+ *
+ * @return Attempt number of the subtask.
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getAttemptNumber() {
+ return getTaskInfo().getAttemptNumber();
+ }
+
+ /**
+ * Returns id of the restored checkpoint, if state was restored from the snapshot of a previous
+ * execution.
+ */
+ OptionalLong getRestoredCheckpointId();
+
+ /**
+ * The ID of the current job. Note that Job ID can change in particular upon manual restart. The
+ * returned ID should NOT be used for any job management tasks.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the job should be
+ * provided uniformly by {@link #getJobInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default JobID getJobId() {
+ return getJobInfo().getJobId();
+ }
+
+ /**
+ * Get the meta information of current job.
+ *
+ * @return the job meta information.
+ */
+ @PublicEvolving
+ JobInfo getJobInfo();
+
+ /**
+ * Get the meta information of current task.
+ *
+ * @return the task meta information.
+ */
+ @PublicEvolving
+ TaskInfo getTaskInfo();
+}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java
new file mode 100644
index 000000000000..2211ce47bd6f
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.connector.sink2;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import java.util.Optional;
+import java.util.function.Consumer;
+
+/** The interface exposes some runtime info for creating a {@link SinkWriter}. */
+@Public
+public interface WriterInitContext extends org.apache.flink.api.connector.sink2.InitContext {
+ /**
+ * Gets the {@link UserCodeClassLoader} to load classes that are not in system's classpath, but
+ * are part of the jar file of a user job.
+ *
+ * @see UserCodeClassLoader
+ */
+ UserCodeClassLoader getUserCodeClassLoader();
+
+ /**
+ * Returns the mailbox executor that allows to execute {@link Runnable}s inside the task thread
+ * in between record processing.
+ *
+ *
Note that this method should not be used per-record for performance reasons in the same
+ * way as records should not be sent to the external system individually. Rather, implementers
+ * are expected to batch records and only enqueue a single {@link Runnable} per batch to handle
+ * the result.
+ */
+ MailboxExecutor getMailboxExecutor();
+
+ /**
+ * Returns a {@link ProcessingTimeService} that can be used to get the current time and register
+ * timers.
+ */
+ ProcessingTimeService getProcessingTimeService();
+
+ /** @return The metric group this writer belongs to. */
+ SinkWriterMetricGroup metricGroup();
+
+ /** Provides a view on this context as a {@link SerializationSchema.InitializationContext}. */
+ SerializationSchema.InitializationContext asSerializationSchemaInitializationContext();
+
+ /** Returns whether object reuse has been enabled or disabled. */
+ boolean isObjectReuseEnabled();
+
+ /** Creates a serializer for the type of sink's input. */
+ TypeSerializer createInputSerializer();
+
+ /**
+ * Returns a metadata consumer, the {@link SinkWriter} can publish metadata events of type
+ * {@link MetaT} to the consumer.
+ *
+ * It is recommended to use a separate thread pool to publish the metadata because enqueuing
+ * a lot of these messages in the mailbox may lead to a performance decrease. thread, and the
+ * {@link Consumer#accept} method is executed very fast.
+ */
+ default Optional> metadataConsumer() {
+ return Optional.empty();
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
new file mode 100644
index 000000000000..856703c42d15
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+import org.apache.flink.configuration.description.TextElement;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+
+import java.time.Duration;
+
+/** A collection of all configuration options that relate to checkpoints and savepoints. */
+public class CheckpointingOptions {
+
+ // ------------------------------------------------------------------------
+ // general checkpoint options
+ // ------------------------------------------------------------------------
+
+ /**
+ * The checkpoint storage used to store operator state locally within the cluster during
+ * execution.
+ *
+ * The implementation can be specified either via their shortcut name, or via the class name
+ * of a {@code StateBackendFactory}. If a StateBackendFactory class name is specified, the
+ * factory is instantiated (via its zero-argument constructor) and its {@code
+ * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called.
+ *
+ *
Recognized shortcut names are 'hashmap' and 'rocksdb'.
+ *
+ * @deprecated Use {@link StateBackendOptions#STATE_BACKEND}.
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS)
+ @Documentation.ExcludeFromDocumentation("Hidden for deprecated")
+ @Deprecated
+ public static final ConfigOption STATE_BACKEND =
+ ConfigOptions.key("state.backend")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text("The state backend to be used to store state.")
+ .linebreak()
+ .text(
+ "The implementation can be specified either via their shortcut "
+ + " name, or via the class name of a %s. "
+ + "If a factory is specified it is instantiated via its "
+ + "zero argument constructor and its %s "
+ + "method is called.",
+ TextElement.code("StateBackendFactory"),
+ TextElement.code(
+ "StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text("Recognized shortcut names are 'hashmap' and 'rocksdb'.")
+ .build());
+
+ /**
+ * The checkpoint storage used to checkpoint state for recovery.
+ *
+ * The implementation can be specified either via their shortcut name, or via the class name
+ * of a {@code CheckpointStorageFactory}. If a CheckpointStorageFactory class name is specified,
+ * the factory is instantiated (via its zero-argument constructor) and its {@code
+ * CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called.
+ *
+ *
Recognized shortcut names are 'jobmanager' and 'filesystem'.
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
+ public static final ConfigOption CHECKPOINT_STORAGE =
+ ConfigOptions.key("state.checkpoint-storage")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The checkpoint storage implementation to be used to checkpoint state.")
+ .linebreak()
+ .text(
+ "The implementation can be specified either via their shortcut "
+ + " name, or via the class name of a %s. "
+ + "If a factory is specified it is instantiated via its "
+ + "zero argument constructor and its %s "
+ + " method is called.",
+ TextElement.code("CheckpointStorageFactory"),
+ TextElement.code(
+ "CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text(
+ "Recognized shortcut names are 'jobmanager' and 'filesystem'.")
+ .build());
+
+ /** The maximum number of completed checkpoints to retain. */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption MAX_RETAINED_CHECKPOINTS =
+ ConfigOptions.key("state.checkpoints.num-retained")
+ .intType()
+ .defaultValue(1)
+ .withDescription("The maximum number of completed checkpoints to retain.");
+
+ /** @deprecated Checkpoints are always asynchronous. */
+ @Deprecated
+ public static final ConfigOption ASYNC_SNAPSHOTS =
+ ConfigOptions.key("state.backend.async")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription("Deprecated option. All state snapshots are asynchronous.");
+
+ /**
+ * Option whether the state backend should create incremental checkpoints, if possible. For an
+ * incremental checkpoint, only a diff from the previous checkpoint is stored, rather than the
+ * complete checkpoint state.
+ *
+ * Once enabled, the state size shown in web UI or fetched from rest API only represents the
+ * delta checkpoint size instead of full checkpoint size.
+ *
+ *
Some state backends may not support incremental checkpoints and ignore this option.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption INCREMENTAL_CHECKPOINTS =
+ ConfigOptions.key("state.backend.incremental")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Option whether the state backend should create incremental checkpoints, if possible. For"
+ + " an incremental checkpoint, only a diff from the previous checkpoint is stored, rather than the"
+ + " complete checkpoint state. Once enabled, the state size shown in web UI or fetched from rest API"
+ + " only represents the delta checkpoint size instead of full checkpoint size."
+ + " Some state backends may not support incremental checkpoints and ignore this option.");
+
+ /**
+ * This option configures local recovery for this state backend. By default, local recovery is
+ * deactivated.
+ *
+ * Local recovery currently only covers keyed state backends (including both the
+ * EmbeddedRocksDBStateBackend and the HashMapStateBackend).
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption LOCAL_RECOVERY =
+ ConfigOptions.key("state.backend.local-recovery")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "This option configures local recovery for this state backend. By default, local recovery is "
+ + "deactivated. Local recovery currently only covers keyed state backends "
+ + "(including both the EmbeddedRocksDBStateBackend and the HashMapStateBackend).");
+
+ /**
+ * The config parameter defining the root directories for storing file-based state for local
+ * recovery.
+ *
+ * Local recovery currently only covers keyed state backends. Currently, MemoryStateBackend
+ * does not support local recovery and ignore this option.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS =
+ ConfigOptions.key("taskmanager.state.local.root-dirs")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The config parameter defining the root directories for storing file-based "
+ + "state for local recovery. Local recovery currently only covers keyed "
+ + "state backends. If not configured it will default to /localState. "
+ + "The can be configured via %s",
+ TextElement.code(
+ ClusterOptions
+ .TASK_MANAGER_PROCESS_WORKING_DIR_BASE
+ .key()))
+ .build());
+
+ // ------------------------------------------------------------------------
+ // Options specific to the file-system-based state backends
+ // ------------------------------------------------------------------------
+
+ /**
+ * The default directory for savepoints. Used by the state backends that write savepoints to
+ * file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 3)
+ public static final ConfigOption SAVEPOINT_DIRECTORY =
+ ConfigOptions.key("state.savepoints.dir")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("savepoints.state.backend.fs.dir")
+ .withDescription(
+ "The default directory for savepoints. Used by the state backends that write savepoints to"
+ + " file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).");
+
+ /**
+ * The default directory used for storing the data files and meta data of checkpoints in a Flink
+ * supported filesystem. The storage path must be accessible from all participating
+ * processes/nodes(i.e. all TaskManagers and JobManagers).
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
+ public static final ConfigOption CHECKPOINTS_DIRECTORY =
+ ConfigOptions.key("state.checkpoints.dir")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("state.backend.fs.checkpointdir")
+ .withDescription(
+ "The default directory used for storing the data files and meta data of checkpoints "
+ + "in a Flink supported filesystem. The storage path must be accessible from all participating processes/nodes"
+ + "(i.e. all TaskManagers and JobManagers).");
+
+ /**
+ * The minimum size of state data files. All state chunks smaller than that are stored inline in
+ * the root checkpoint metadata file.
+ */
+ @Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS)
+ public static final ConfigOption FS_SMALL_FILE_THRESHOLD =
+ ConfigOptions.key("state.storage.fs.memory-threshold")
+ .memoryType()
+ .defaultValue(MemorySize.parse("20kb"))
+ .withDescription(
+ "The minimum size of state data files. All state chunks smaller than that are stored"
+ + " inline in the root checkpoint metadata file. The max memory threshold for this configuration is 1MB.")
+ .withDeprecatedKeys("state.backend.fs.memory-threshold");
+
+ /**
+ * The default size of the write buffer for the checkpoint streams that write to file systems.
+ */
+ @Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS)
+ public static final ConfigOption FS_WRITE_BUFFER_SIZE =
+ ConfigOptions.key("state.storage.fs.write-buffer-size")
+ .intType()
+ .defaultValue(4 * 1024)
+ .withDescription(
+ String.format(
+ "The default size of the write buffer for the checkpoint streams that write to file systems. "
+ + "The actual write buffer size is determined to be the maximum of the value of this option and option '%s'.",
+ FS_SMALL_FILE_THRESHOLD.key()))
+ .withDeprecatedKeys("state.backend.fs.write-buffer-size");
+
+ public static final ConfigOption CHECKPOINTING_INTERVAL =
+ ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+
+ public static final ConfigOption ENABLE_UNALIGNED =
+ ExecutionCheckpointingOptions.ENABLE_UNALIGNED;
+
+ public static final ConfigOption CHECKPOINTING_CONSISTENCY_MODE =
+ ExecutionCheckpointingOptions.CHECKPOINTING_MODE;
+
+ public static final ConfigOption MAX_CONCURRENT_CHECKPOINTS =
+ ExecutionCheckpointingOptions.MAX_CONCURRENT_CHECKPOINTS;
+
+ public static final ConfigOption CHECKPOINTING_TIMEOUT =
+ ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT;
+}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java
new file mode 100644
index 000000000000..2193c2a6aab9
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The {@link ConfigOption configuration options} used when restoring state from a savepoint or a
+ * checkpoint.
+ */
+@PublicEvolving
+public class StateRecoveryOptions {
+ /** The path to a savepoint that will be used to bootstrap the pipeline's state. */
+ public static final ConfigOption SAVEPOINT_PATH =
+ key("execution.state-recovery.path")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("execution.savepoint.path")
+ .withDescription(
+ "Path to a savepoint to restore the job from (for example hdfs:///flink/savepoint-1537).");
+}
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
new file mode 100644
index 000000000000..fc7eb0d48356
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.sink.v2;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+
+import java.io.IOException;
+
+/**
+ * A special sink that ignores all elements.
+ *
+ * @param The type of elements received by the sink.
+ */
+@PublicEvolving
+public class DiscardingSink implements Sink, SupportsConcurrentExecutionAttempts {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public SinkWriter createWriter(InitContext context) throws IOException {
+ return new DiscardingElementWriter();
+ }
+
+ private class DiscardingElementWriter implements SinkWriter {
+
+ @Override
+ public void write(IN element, Context context) throws IOException, InterruptedException {
+ // discard it.
+ }
+
+ @Override
+ public void flush(boolean endOfInput) throws IOException, InterruptedException {
+ // this writer has no pending data.
+ }
+
+ @Override
+ public void close() throws Exception {
+ // do nothing.
+ }
+ }
+}
diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java
similarity index 98%
rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java
rename to paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java
index 3805f6f8c536..b38e32559c36 100644
--- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java
@@ -77,12 +77,12 @@
* pass the watermark recorded in the snapshot.
*
*/
-public class MonitorFunction extends RichSourceFunction
+public class MonitorSource extends RichSourceFunction
implements CheckpointedFunction, CheckpointListener {
private static final long serialVersionUID = 1L;
- private static final Logger LOG = LoggerFactory.getLogger(MonitorFunction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class);
private final ReadBuilder readBuilder;
private final long monitorInterval;
@@ -97,7 +97,7 @@ public class MonitorFunction extends RichSourceFunction
private transient ListState> nextSnapshotState;
private transient TreeMap nextSnapshotPerCheckpoint;
- public MonitorFunction(
+ public MonitorSource(
ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) {
this.readBuilder = readBuilder;
this.monitorInterval = monitorInterval;
@@ -238,7 +238,7 @@ public static DataStream buildSource(
BucketMode bucketMode) {
SingleOutputStreamOperator singleOutputStreamOperator =
env.addSource(
- new MonitorFunction(
+ new MonitorSource(
readBuilder, monitorInterval, emitSnapshotWatermark),
name + "-Monitor",
new JavaTypeInfo<>(Split.class))
diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java
new file mode 100644
index 000000000000..460fea55ad7a
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.utils;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+/** Utility methods about Flink runtime context to resolve compatibility issues. */
+public class RuntimeContextUtils {
+ public static int getNumberOfParallelSubtasks(RuntimeContext context) {
+ return context.getNumberOfParallelSubtasks();
+ }
+
+ public static int getIndexOfThisSubtask(RuntimeContext context) {
+ return context.getIndexOfThisSubtask();
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java
new file mode 100644
index 000000000000..698900436e8d
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.sink.cdc;
+
+import org.apache.paimon.types.BigIntType;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.RowKind;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.types.VarCharType;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link RichCdcMultiplexRecord} be deserialized by flink KryoSerializer. */
+public class CdcRecordSerializeITCase {
+
+ @Test
+ public void testCdcRecordKryoSerialize() throws IOException {
+ KryoSerializer kr =
+ createFlinkKryoSerializer(RichCdcMultiplexRecord.class);
+ RowType.Builder rowType = RowType.builder();
+ rowType.field("id", new BigIntType());
+ rowType.field("name", new VarCharType());
+ rowType.field("pt", new VarCharType());
+ // this is an unmodifiable list.
+ List fields = rowType.build().getFields();
+ List primaryKeys = Collections.singletonList("id");
+ Map recordData = new HashMap<>();
+ recordData.put("id", "1");
+ recordData.put("name", "HunterXHunter");
+ recordData.put("pt", "2024-06-28");
+ CdcRecord cdcRecord = new CdcRecord(RowKind.INSERT, recordData);
+ RichCdcMultiplexRecord serializeRecord =
+ new RichCdcMultiplexRecord("default", "T", fields, primaryKeys, cdcRecord);
+
+ TestOutputView outputView = new TestOutputView();
+ kr.serialize(serializeRecord, outputView);
+ RichCdcMultiplexRecord deserializeRecord = kr.deserialize(outputView.getInputView());
+ assertThat(deserializeRecord.toRichCdcRecord().toCdcRecord()).isEqualTo(cdcRecord);
+ assertThat(deserializeRecord.databaseName()).isEqualTo("default");
+ assertThat(deserializeRecord.tableName()).isEqualTo("T");
+ assertThat(deserializeRecord.primaryKeys()).isEqualTo(primaryKeys);
+ assertThat(deserializeRecord.fields()).isEqualTo(fields);
+ }
+
+ @Test
+ public void testUnmodifiableListKryoSerialize() throws IOException {
+ KryoSerializer kryoSerializer = createFlinkKryoSerializer(List.class);
+ RowType.Builder rowType = RowType.builder();
+ rowType.field("id", new BigIntType());
+ rowType.field("name", new VarCharType());
+ rowType.field("pt", new VarCharType());
+ // Deserializing an unmodifiable list would be throw
+ // java.lang.UnsupportedOperationException.
+ List fields = rowType.build().getFields();
+
+ TestOutputView outputView = new TestOutputView();
+ kryoSerializer.serialize(fields, outputView);
+ assertThatThrownBy(() -> kryoSerializer.deserialize(outputView.getInputView()))
+ .satisfies(anyCauseMatches(UnsupportedOperationException.class));
+
+ // This `fields` is a modifiable list should be successfully serialized.
+ TestOutputView outputView2 = new TestOutputView();
+ fields = new ArrayList<>(fields);
+ kryoSerializer.serialize(fields, outputView2);
+ List deserializeRecord = kryoSerializer.deserialize(outputView2.getInputView());
+ assertThat(deserializeRecord).isEqualTo(fields);
+ }
+
+ public static KryoSerializer createFlinkKryoSerializer(Class type) {
+ return new KryoSerializer<>(type, new ExecutionConfig());
+ }
+
+ private static final class TestOutputView extends DataOutputStream implements DataOutputView {
+
+ public TestOutputView() {
+ super(new ByteArrayOutputStream(4096));
+ }
+
+ public TestInputView getInputView() {
+ ByteArrayOutputStream baos = (ByteArrayOutputStream) out;
+ return new TestInputView(baos.toByteArray());
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ for (int i = 0; i < numBytes; i++) {
+ write(0);
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ byte[] buffer = new byte[numBytes];
+ source.readFully(buffer);
+ write(buffer);
+ }
+ }
+
+ private static final class TestInputView extends DataInputStream implements DataInputView {
+
+ public TestInputView(byte[] data) {
+ super(new ByteArrayInputStream(data));
+ }
+
+ @Override
+ public void skipBytesToRead(int numBytes) throws IOException {
+ while (numBytes > 0) {
+ int skipped = skipBytes(numBytes);
+ numBytes -= skipped;
+ }
+ }
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/JobInfo.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/JobInfo.java
new file mode 100644
index 000000000000..38e3bfea6965
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/JobInfo.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** The {@link JobInfo} represents the meta information of current job. */
+@PublicEvolving
+public interface JobInfo {
+
+ /**
+ * Get the ID of the job.
+ *
+ * @return the ID of the job
+ */
+ JobID getJobId();
+
+ /**
+ * Get the name of the job.
+ *
+ * @return the name of the job
+ */
+ String getJobName();
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/SupportsConcurrentExecutionAttempts.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/SupportsConcurrentExecutionAttempts.java
new file mode 100644
index 000000000000..a8a61d72883a
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/SupportsConcurrentExecutionAttempts.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * The interface indicates that it supports multiple attempts executing at the same time.
+ *
+ * Currently, the interface is used for speculative execution. If a sink implementation (SinkV2,
+ * OutputFormat or SinkFunction) inherits this interface, the sink operator would be considered to
+ * support speculative execution.
+ */
+@PublicEvolving
+public interface SupportsConcurrentExecutionAttempts {}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java
new file mode 100644
index 000000000000..21fca4e4c319
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** The default implementation of {@link OpenContext}. */
+@PublicEvolving
+public class DefaultOpenContext implements OpenContext {
+
+ public static final OpenContext INSTANCE = new DefaultOpenContext();
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
new file mode 100644
index 000000000000..4ff5484b3b08
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * The {@link OpenContext} interface provides necessary information required by the {@link
+ * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be
+ * used to add more methods without affecting the signature of {@code RichFunction#open}.
+ */
+@PublicEvolving
+public interface OpenContext {}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/RichFunction.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
new file mode 100644
index 000000000000..ae83fb30f2bd
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * An base interface for all rich user-defined functions. This class defines methods for the life
+ * cycle of the functions, as well as methods to access the context in which the functions are
+ * executed.
+ */
+@Public
+public interface RichFunction extends Function {
+
+ /**
+ * Initialization method for the function. It is called before the actual working methods (like
+ * map or join) and thus suitable for one time setup work. For functions that are
+ * part of an iteration, this method will be invoked at the beginning of each iteration
+ * superstep.
+ *
+ *
The configuration object passed to the function can be used for configuration and
+ * initialization. The configuration contains all parameters that were configured on the
+ * function in the program composition.
+ *
+ *
{@code
+ * public class MyFilter extends RichFilterFunction {
+ *
+ * private String searchString;
+ *
+ * public void open(Configuration parameters) {
+ * this.searchString = parameters.getString("foo");
+ * }
+ *
+ * public boolean filter(String value) {
+ * return value.equals(searchString);
+ * }
+ * }
+ * }
+ *
+ * By default, this method does nothing.
+ *
+ * @param parameters The configuration containing the parameters attached to the contract.
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ * @see org.apache.flink.configuration.Configuration
+ * @deprecated This method is deprecated since Flink 1.19. The users are recommended to
+ * implement {@code open(OpenContext openContext)} and implement {@code open(Configuration
+ * parameters)} with an empty body instead. 1. If you implement {@code open(OpenContext
+ * openContext)}, the {@code open(OpenContext openContext)} will be invoked and the {@code
+ * open(Configuration parameters)} won't be invoked. 2. If you don't implement {@code
+ * open(OpenContext openContext)}, the {@code open(Configuration parameters)} will be
+ * invoked in the default implementation of the {@code open(OpenContext openContext)}.
+ * @see
+ * FLIP-344: Remove parameter in RichFunction#open
+ */
+ @Deprecated
+ void open(Configuration parameters) throws Exception;
+
+ /**
+ * Initialization method for the function. It is called before the actual working methods (like
+ * map or join) and thus suitable for one time setup work. For functions that are
+ * part of an iteration, this method will be invoked at the beginning of each iteration
+ * superstep.
+ *
+ *
The openContext object passed to the function can be used for configuration and
+ * initialization. The openContext contains some necessary information that were configured on
+ * the function in the program composition.
+ *
+ *
{@code
+ * public class MyFilter extends RichFilterFunction {
+ *
+ * private String searchString;
+ *
+ * public void open(OpenContext openContext) {
+ * // initialize the value of searchString
+ * }
+ *
+ * public boolean filter(String value) {
+ * return value.equals(searchString);
+ * }
+ * }
+ * }
+ *
+ * By default, this method does nothing.
+ *
+ *
1. If you implement {@code open(OpenContext openContext)}, the {@code open(OpenContext
+ * openContext)} will be invoked and the {@code open(Configuration parameters)} won't be
+ * invoked. 2. If you don't implement {@code open(OpenContext openContext)}, the {@code
+ * open(Configuration parameters)} will be invoked in the default implementation of the {@code
+ * open(OpenContext openContext)}.
+ *
+ * @param openContext The context containing information about the context in which the function
+ * is opened.
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ */
+ @PublicEvolving
+ default void open(OpenContext openContext) throws Exception {
+ open(new Configuration());
+ }
+
+ /**
+ * Tear-down method for the user code. It is called after the last call to the main working
+ * methods (e.g. map or join). For functions that are part of an iteration, this
+ * method will be invoked after each iteration superstep.
+ *
+ *
This method can be used for clean up work.
+ *
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ */
+ void close() throws Exception;
+
+ // ------------------------------------------------------------------------
+ // Runtime context
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the context that contains information about the UDF's runtime, such as the parallelism
+ * of the function, the subtask index of the function, or the name of the task that executes the
+ * function.
+ *
+ *
The RuntimeContext also gives access to the {@link
+ * org.apache.flink.api.common.accumulators.Accumulator}s and the {@link
+ * org.apache.flink.api.common.cache.DistributedCache}.
+ *
+ * @return The UDF's runtime context.
+ */
+ RuntimeContext getRuntimeContext();
+
+ /**
+ * Gets a specialized version of the {@link RuntimeContext}, which has additional information
+ * about the iteration in which the function is executed. This IterationRuntimeContext is only
+ * available if the function is part of an iteration. Otherwise, this method throws an
+ * exception.
+ *
+ * @return The IterationRuntimeContext.
+ * @throws java.lang.IllegalStateException Thrown, if the function is not executed as part of an
+ * iteration.
+ */
+ IterationRuntimeContext getIterationRuntimeContext();
+
+ /**
+ * Sets the function's runtime context. Called by the framework when creating a parallel
+ * instance of the function.
+ *
+ * @param t The runtime context.
+ */
+ void setRuntimeContext(RuntimeContext t);
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java
new file mode 100644
index 000000000000..5e13a9b62329
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.serialization;
+
+import com.esotericsoftware.kryo.Serializer;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInfoFactory;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.util.TernaryBoolean;
+
+import java.io.Serializable;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+
+/**
+ * A config to define the behavior for serializers in Flink job, it manages the registered types and
+ * serializers. The config is created from job configuration and used by Flink to create serializers
+ * for data types.
+ */
+@PublicEvolving
+public interface SerializerConfig extends Serializable {
+ /**
+ * Adds a new Kryo default serializer to the Runtime.
+ *
+ *
Note that the serializer instance must be serializable (as defined by
+ * java.io.Serializable), because it may be distributed to the worker nodes by java
+ * serialization.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializer The serializer to use.
+ */
+ @Internal
+ & Serializable> void addDefaultKryoSerializer(
+ Class> type, T serializer);
+
+ /**
+ * Adds a new Kryo default serializer to the Runtime.
+ *
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializerClass The class of the serializer to use.
+ */
+ @Internal
+ void addDefaultKryoSerializer(Class> type, Class extends Serializer>> serializerClass);
+
+ /**
+ * Registers the given type with a Kryo Serializer.
+ *
+ *
Note that the serializer instance must be serializable (as defined by
+ * java.io.Serializable), because it may be distributed to the worker nodes by java
+ * serialization.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializer The serializer to use.
+ */
+ @Internal
+ & Serializable> void registerTypeWithKryoSerializer(
+ Class> type, T serializer);
+
+ /**
+ * Registers the given Serializer via its class as a serializer for the given type at the
+ * KryoSerializer.
+ *
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializerClass The class of the serializer to use.
+ */
+ @Internal
+ @SuppressWarnings("rawtypes")
+ void registerTypeWithKryoSerializer(Class> type, Class extends Serializer> serializerClass);
+
+ /**
+ * Registers the given type with the serialization stack. If the type is eventually serialized
+ * as a POJO, then the type is registered with the POJO serializer. If the type ends up being
+ * serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
+ * written.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the type to register.
+ */
+ @Internal
+ void registerPojoType(Class> type);
+
+ /**
+ * Registers the given type with the serialization stack. If the type is eventually serialized
+ * as a POJO, then the type is registered with the POJO serializer. If the type ends up being
+ * serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
+ * written.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the type to register.
+ */
+ @Internal
+ void registerKryoType(Class> type);
+
+ /**
+ * Returns the registered types with Kryo Serializers.
+ *
+ * @deprecated The method is deprecated because instance-type Kryo serializer definition based
+ * on {@link ExecutionConfig.SerializableSerializer} is deprecated. Use class-type Kryo
+ * serializers instead.
+ */
+ @Deprecated
+ LinkedHashMap, ExecutionConfig.SerializableSerializer>>
+ getRegisteredTypesWithKryoSerializers();
+
+ /** Returns the registered types with their Kryo Serializer classes. */
+ LinkedHashMap, Class extends Serializer>>>
+ getRegisteredTypesWithKryoSerializerClasses();
+
+ /**
+ * Returns the registered default Kryo Serializers.
+ *
+ * @deprecated The method is deprecated because {@link ExecutionConfig.SerializableSerializer}
+ * is deprecated.
+ */
+ @Deprecated
+ LinkedHashMap, ExecutionConfig.SerializableSerializer>> getDefaultKryoSerializers();
+
+ /** Returns the registered default Kryo Serializer classes. */
+ LinkedHashMap, Class extends Serializer>>> getDefaultKryoSerializerClasses();
+
+ /** Returns the registered Kryo types. */
+ LinkedHashSet> getRegisteredKryoTypes();
+
+ /** Returns the registered POJO types. */
+ LinkedHashSet> getRegisteredPojoTypes();
+
+ /** Returns the registered type info factories. */
+ Map, Class extends TypeInfoFactory>>> getRegisteredTypeInfoFactories();
+
+ /**
+ * Checks whether generic types are supported. Generic types are types that go through Kryo
+ * during serialization.
+ *
+ * Generic types are enabled by default.
+ */
+ boolean hasGenericTypesDisabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setGenericTypes(boolean genericTypes);
+
+ /** Returns whether Kryo is the serializer for POJOs. */
+ boolean isForceKryoEnabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setForceKryo(boolean forceKryo);
+
+ /** Returns whether the Apache Avro is the serializer for POJOs. */
+ boolean isForceAvroEnabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setForceAvro(boolean forceAvro);
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ public void setForceKryoAvro(boolean forceKryoAvro);
+
+ /** Returns whether forces Flink to register Apache Avro classes in Kryo serializer. */
+ TernaryBoolean isForceKryoAvroEnabled();
+
+ /**
+ * Sets all relevant options contained in the {@link ReadableConfig} such as e.g. {@link
+ * PipelineOptions#FORCE_KRYO}.
+ *
+ *
It will change the value of a setting only if a corresponding option was set in the {@code
+ * configuration}. If a key is not present, the current value of a field will remain untouched.
+ *
+ * @param configuration a configuration to read the values from
+ * @param classLoader a class loader to use when loading classes
+ */
+ void configure(ReadableConfig configuration, ClassLoader classLoader);
+
+ SerializerConfig copy();
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java
new file mode 100644
index 000000000000..ae637d6456e7
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.connector.sink2;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobInfo;
+import org.apache.flink.api.common.TaskInfo;
+
+import java.util.OptionalLong;
+
+/**
+ * Common interface which exposes runtime info for creating {@link SinkWriter} and {@link Committer}
+ * objects.
+ */
+@Internal
+public interface InitContext {
+ /**
+ * The first checkpoint id when an application is started and not recovered from a previously
+ * taken checkpoint or savepoint.
+ */
+ long INITIAL_CHECKPOINT_ID = 1;
+
+ /**
+ * Get the id of task where the committer is running.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getSubtaskId() {
+ return getTaskInfo().getIndexOfThisSubtask();
+ }
+
+ /**
+ * Get the number of parallel committer tasks.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getNumberOfParallelSubtasks() {
+ return getTaskInfo().getNumberOfParallelSubtasks();
+ }
+
+ /**
+ * Gets the attempt number of this parallel subtask. First attempt is numbered 0.
+ *
+ * @return Attempt number of the subtask.
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getAttemptNumber() {
+ return getTaskInfo().getAttemptNumber();
+ }
+
+ /**
+ * Returns id of the restored checkpoint, if state was restored from the snapshot of a previous
+ * execution.
+ */
+ OptionalLong getRestoredCheckpointId();
+
+ /**
+ * The ID of the current job. Note that Job ID can change in particular upon manual restart. The
+ * returned ID should NOT be used for any job management tasks.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the job should be
+ * provided uniformly by {@link #getJobInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default JobID getJobId() {
+ return getJobInfo().getJobId();
+ }
+
+ /**
+ * Get the meta information of current job.
+ *
+ * @return the job meta information.
+ */
+ @PublicEvolving
+ JobInfo getJobInfo();
+
+ /**
+ * Get the meta information of current task.
+ *
+ * @return the task meta information.
+ */
+ @PublicEvolving
+ TaskInfo getTaskInfo();
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java
new file mode 100644
index 000000000000..2211ce47bd6f
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.connector.sink2;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import java.util.Optional;
+import java.util.function.Consumer;
+
+/** The interface exposes some runtime info for creating a {@link SinkWriter}. */
+@Public
+public interface WriterInitContext extends org.apache.flink.api.connector.sink2.InitContext {
+ /**
+ * Gets the {@link UserCodeClassLoader} to load classes that are not in system's classpath, but
+ * are part of the jar file of a user job.
+ *
+ * @see UserCodeClassLoader
+ */
+ UserCodeClassLoader getUserCodeClassLoader();
+
+ /**
+ * Returns the mailbox executor that allows to execute {@link Runnable}s inside the task thread
+ * in between record processing.
+ *
+ *
Note that this method should not be used per-record for performance reasons in the same
+ * way as records should not be sent to the external system individually. Rather, implementers
+ * are expected to batch records and only enqueue a single {@link Runnable} per batch to handle
+ * the result.
+ */
+ MailboxExecutor getMailboxExecutor();
+
+ /**
+ * Returns a {@link ProcessingTimeService} that can be used to get the current time and register
+ * timers.
+ */
+ ProcessingTimeService getProcessingTimeService();
+
+ /** @return The metric group this writer belongs to. */
+ SinkWriterMetricGroup metricGroup();
+
+ /** Provides a view on this context as a {@link SerializationSchema.InitializationContext}. */
+ SerializationSchema.InitializationContext asSerializationSchemaInitializationContext();
+
+ /** Returns whether object reuse has been enabled or disabled. */
+ boolean isObjectReuseEnabled();
+
+ /** Creates a serializer for the type of sink's input. */
+ TypeSerializer createInputSerializer();
+
+ /**
+ * Returns a metadata consumer, the {@link SinkWriter} can publish metadata events of type
+ * {@link MetaT} to the consumer.
+ *
+ * It is recommended to use a separate thread pool to publish the metadata because enqueuing
+ * a lot of these messages in the mailbox may lead to a performance decrease. thread, and the
+ * {@link Consumer#accept} method is executed very fast.
+ */
+ default Optional> metadataConsumer() {
+ return Optional.empty();
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
new file mode 100644
index 000000000000..856703c42d15
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+import org.apache.flink.configuration.description.TextElement;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+
+import java.time.Duration;
+
+/** A collection of all configuration options that relate to checkpoints and savepoints. */
+public class CheckpointingOptions {
+
+ // ------------------------------------------------------------------------
+ // general checkpoint options
+ // ------------------------------------------------------------------------
+
+ /**
+ * The checkpoint storage used to store operator state locally within the cluster during
+ * execution.
+ *
+ * The implementation can be specified either via their shortcut name, or via the class name
+ * of a {@code StateBackendFactory}. If a StateBackendFactory class name is specified, the
+ * factory is instantiated (via its zero-argument constructor) and its {@code
+ * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called.
+ *
+ *
Recognized shortcut names are 'hashmap' and 'rocksdb'.
+ *
+ * @deprecated Use {@link StateBackendOptions#STATE_BACKEND}.
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS)
+ @Documentation.ExcludeFromDocumentation("Hidden for deprecated")
+ @Deprecated
+ public static final ConfigOption STATE_BACKEND =
+ ConfigOptions.key("state.backend")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text("The state backend to be used to store state.")
+ .linebreak()
+ .text(
+ "The implementation can be specified either via their shortcut "
+ + " name, or via the class name of a %s. "
+ + "If a factory is specified it is instantiated via its "
+ + "zero argument constructor and its %s "
+ + "method is called.",
+ TextElement.code("StateBackendFactory"),
+ TextElement.code(
+ "StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text("Recognized shortcut names are 'hashmap' and 'rocksdb'.")
+ .build());
+
+ /**
+ * The checkpoint storage used to checkpoint state for recovery.
+ *
+ * The implementation can be specified either via their shortcut name, or via the class name
+ * of a {@code CheckpointStorageFactory}. If a CheckpointStorageFactory class name is specified,
+ * the factory is instantiated (via its zero-argument constructor) and its {@code
+ * CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called.
+ *
+ *
Recognized shortcut names are 'jobmanager' and 'filesystem'.
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
+ public static final ConfigOption CHECKPOINT_STORAGE =
+ ConfigOptions.key("state.checkpoint-storage")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The checkpoint storage implementation to be used to checkpoint state.")
+ .linebreak()
+ .text(
+ "The implementation can be specified either via their shortcut "
+ + " name, or via the class name of a %s. "
+ + "If a factory is specified it is instantiated via its "
+ + "zero argument constructor and its %s "
+ + " method is called.",
+ TextElement.code("CheckpointStorageFactory"),
+ TextElement.code(
+ "CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text(
+ "Recognized shortcut names are 'jobmanager' and 'filesystem'.")
+ .build());
+
+ /** The maximum number of completed checkpoints to retain. */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption MAX_RETAINED_CHECKPOINTS =
+ ConfigOptions.key("state.checkpoints.num-retained")
+ .intType()
+ .defaultValue(1)
+ .withDescription("The maximum number of completed checkpoints to retain.");
+
+ /** @deprecated Checkpoints are always asynchronous. */
+ @Deprecated
+ public static final ConfigOption ASYNC_SNAPSHOTS =
+ ConfigOptions.key("state.backend.async")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription("Deprecated option. All state snapshots are asynchronous.");
+
+ /**
+ * Option whether the state backend should create incremental checkpoints, if possible. For an
+ * incremental checkpoint, only a diff from the previous checkpoint is stored, rather than the
+ * complete checkpoint state.
+ *
+ * Once enabled, the state size shown in web UI or fetched from rest API only represents the
+ * delta checkpoint size instead of full checkpoint size.
+ *
+ *
Some state backends may not support incremental checkpoints and ignore this option.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption INCREMENTAL_CHECKPOINTS =
+ ConfigOptions.key("state.backend.incremental")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Option whether the state backend should create incremental checkpoints, if possible. For"
+ + " an incremental checkpoint, only a diff from the previous checkpoint is stored, rather than the"
+ + " complete checkpoint state. Once enabled, the state size shown in web UI or fetched from rest API"
+ + " only represents the delta checkpoint size instead of full checkpoint size."
+ + " Some state backends may not support incremental checkpoints and ignore this option.");
+
+ /**
+ * This option configures local recovery for this state backend. By default, local recovery is
+ * deactivated.
+ *
+ * Local recovery currently only covers keyed state backends (including both the
+ * EmbeddedRocksDBStateBackend and the HashMapStateBackend).
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption LOCAL_RECOVERY =
+ ConfigOptions.key("state.backend.local-recovery")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "This option configures local recovery for this state backend. By default, local recovery is "
+ + "deactivated. Local recovery currently only covers keyed state backends "
+ + "(including both the EmbeddedRocksDBStateBackend and the HashMapStateBackend).");
+
+ /**
+ * The config parameter defining the root directories for storing file-based state for local
+ * recovery.
+ *
+ * Local recovery currently only covers keyed state backends. Currently, MemoryStateBackend
+ * does not support local recovery and ignore this option.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS =
+ ConfigOptions.key("taskmanager.state.local.root-dirs")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The config parameter defining the root directories for storing file-based "
+ + "state for local recovery. Local recovery currently only covers keyed "
+ + "state backends. If not configured it will default to /localState. "
+ + "The can be configured via %s",
+ TextElement.code(
+ ClusterOptions
+ .TASK_MANAGER_PROCESS_WORKING_DIR_BASE
+ .key()))
+ .build());
+
+ // ------------------------------------------------------------------------
+ // Options specific to the file-system-based state backends
+ // ------------------------------------------------------------------------
+
+ /**
+ * The default directory for savepoints. Used by the state backends that write savepoints to
+ * file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 3)
+ public static final ConfigOption SAVEPOINT_DIRECTORY =
+ ConfigOptions.key("state.savepoints.dir")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("savepoints.state.backend.fs.dir")
+ .withDescription(
+ "The default directory for savepoints. Used by the state backends that write savepoints to"
+ + " file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).");
+
+ /**
+ * The default directory used for storing the data files and meta data of checkpoints in a Flink
+ * supported filesystem. The storage path must be accessible from all participating
+ * processes/nodes(i.e. all TaskManagers and JobManagers).
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
+ public static final ConfigOption CHECKPOINTS_DIRECTORY =
+ ConfigOptions.key("state.checkpoints.dir")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("state.backend.fs.checkpointdir")
+ .withDescription(
+ "The default directory used for storing the data files and meta data of checkpoints "
+ + "in a Flink supported filesystem. The storage path must be accessible from all participating processes/nodes"
+ + "(i.e. all TaskManagers and JobManagers).");
+
+ /**
+ * The minimum size of state data files. All state chunks smaller than that are stored inline in
+ * the root checkpoint metadata file.
+ */
+ @Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS)
+ public static final ConfigOption FS_SMALL_FILE_THRESHOLD =
+ ConfigOptions.key("state.storage.fs.memory-threshold")
+ .memoryType()
+ .defaultValue(MemorySize.parse("20kb"))
+ .withDescription(
+ "The minimum size of state data files. All state chunks smaller than that are stored"
+ + " inline in the root checkpoint metadata file. The max memory threshold for this configuration is 1MB.")
+ .withDeprecatedKeys("state.backend.fs.memory-threshold");
+
+ /**
+ * The default size of the write buffer for the checkpoint streams that write to file systems.
+ */
+ @Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS)
+ public static final ConfigOption FS_WRITE_BUFFER_SIZE =
+ ConfigOptions.key("state.storage.fs.write-buffer-size")
+ .intType()
+ .defaultValue(4 * 1024)
+ .withDescription(
+ String.format(
+ "The default size of the write buffer for the checkpoint streams that write to file systems. "
+ + "The actual write buffer size is determined to be the maximum of the value of this option and option '%s'.",
+ FS_SMALL_FILE_THRESHOLD.key()))
+ .withDeprecatedKeys("state.backend.fs.write-buffer-size");
+
+ public static final ConfigOption CHECKPOINTING_INTERVAL =
+ ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+
+ public static final ConfigOption ENABLE_UNALIGNED =
+ ExecutionCheckpointingOptions.ENABLE_UNALIGNED;
+
+ public static final ConfigOption CHECKPOINTING_CONSISTENCY_MODE =
+ ExecutionCheckpointingOptions.CHECKPOINTING_MODE;
+
+ public static final ConfigOption MAX_CONCURRENT_CHECKPOINTS =
+ ExecutionCheckpointingOptions.MAX_CONCURRENT_CHECKPOINTS;
+
+ public static final ConfigOption CHECKPOINTING_TIMEOUT =
+ ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT;
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java
new file mode 100644
index 000000000000..2193c2a6aab9
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The {@link ConfigOption configuration options} used when restoring state from a savepoint or a
+ * checkpoint.
+ */
+@PublicEvolving
+public class StateRecoveryOptions {
+ /** The path to a savepoint that will be used to bootstrap the pipeline's state. */
+ public static final ConfigOption SAVEPOINT_PATH =
+ key("execution.state-recovery.path")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("execution.savepoint.path")
+ .withDescription(
+ "Path to a savepoint to restore the job from (for example hdfs:///flink/savepoint-1537).");
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
new file mode 100644
index 000000000000..fc7eb0d48356
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.sink.v2;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+
+import java.io.IOException;
+
+/**
+ * A special sink that ignores all elements.
+ *
+ * @param The type of elements received by the sink.
+ */
+@PublicEvolving
+public class DiscardingSink implements Sink, SupportsConcurrentExecutionAttempts {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public SinkWriter createWriter(InitContext context) throws IOException {
+ return new DiscardingElementWriter();
+ }
+
+ private class DiscardingElementWriter implements SinkWriter {
+
+ @Override
+ public void write(IN element, Context context) throws IOException, InterruptedException {
+ // discard it.
+ }
+
+ @Override
+ public void flush(boolean endOfInput) throws IOException, InterruptedException {
+ // this writer has no pending data.
+ }
+
+ @Override
+ public void close() throws Exception {
+ // do nothing.
+ }
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java
new file mode 100644
index 000000000000..b38e32559c36
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.source.operator;
+
+import org.apache.paimon.flink.utils.JavaTypeInfo;
+import org.apache.paimon.table.BucketMode;
+import org.apache.paimon.table.sink.ChannelComputer;
+import org.apache.paimon.table.source.DataSplit;
+import org.apache.paimon.table.source.EndOfScanException;
+import org.apache.paimon.table.source.ReadBuilder;
+import org.apache.paimon.table.source.Split;
+import org.apache.paimon.table.source.StreamTableScan;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+
+import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE;
+
+/**
+ * This is the single (non-parallel) monitoring task, it is responsible for:
+ *
+ *
+ * - Monitoring snapshots of the Paimon table.
+ *
- Creating the {@link Split splits} corresponding to the incremental files
+ *
- Assigning them to downstream tasks for further processing.
+ *
+ *
+ * The splits to be read are forwarded to the downstream {@link ReadOperator} which can have
+ * parallelism greater than one.
+ *
+ *
Currently, there are two features that rely on this monitor:
+ *
+ *
+ * - Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all
+ * data in a snapshot is consumed within each checkpoint.
+ *
- Snapshot-watermark: when there is no watermark definition, the default Paimon table will
+ * pass the watermark recorded in the snapshot.
+ *
+ */
+public class MonitorSource extends RichSourceFunction
+ implements CheckpointedFunction, CheckpointListener {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class);
+
+ private final ReadBuilder readBuilder;
+ private final long monitorInterval;
+ private final boolean emitSnapshotWatermark;
+
+ private volatile boolean isRunning = true;
+
+ private transient StreamTableScan scan;
+ private transient SourceContext ctx;
+
+ private transient ListState checkpointState;
+ private transient ListState> nextSnapshotState;
+ private transient TreeMap nextSnapshotPerCheckpoint;
+
+ public MonitorSource(
+ ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) {
+ this.readBuilder = readBuilder;
+ this.monitorInterval = monitorInterval;
+ this.emitSnapshotWatermark = emitSnapshotWatermark;
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ this.scan = readBuilder.newStreamScan();
+
+ this.checkpointState =
+ context.getOperatorStateStore()
+ .getListState(
+ new ListStateDescriptor<>(
+ "next-snapshot", LongSerializer.INSTANCE));
+
+ @SuppressWarnings("unchecked")
+ final Class> typedTuple =
+ (Class>) (Class>) Tuple2.class;
+ this.nextSnapshotState =
+ context.getOperatorStateStore()
+ .getListState(
+ new ListStateDescriptor<>(
+ "next-snapshot-per-checkpoint",
+ new TupleSerializer<>(
+ typedTuple,
+ new TypeSerializer[] {
+ LongSerializer.INSTANCE, LongSerializer.INSTANCE
+ })));
+
+ this.nextSnapshotPerCheckpoint = new TreeMap<>();
+
+ if (context.isRestored()) {
+ LOG.info("Restoring state for the {}.", getClass().getSimpleName());
+
+ List retrievedStates = new ArrayList<>();
+ for (Long entry : this.checkpointState.get()) {
+ retrievedStates.add(entry);
+ }
+
+ // given that the parallelism of the function is 1, we can only have 1 retrieved items.
+ Preconditions.checkArgument(
+ retrievedStates.size() <= 1,
+ getClass().getSimpleName() + " retrieved invalid state.");
+
+ if (retrievedStates.size() == 1) {
+ this.scan.restore(retrievedStates.get(0));
+ }
+
+ for (Tuple2 tuple2 : nextSnapshotState.get()) {
+ nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1);
+ }
+ } else {
+ LOG.info("No state to restore for the {}.", getClass().getSimpleName());
+ }
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
+ this.checkpointState.clear();
+ Long nextSnapshot = this.scan.checkpoint();
+ if (nextSnapshot != null) {
+ this.checkpointState.add(nextSnapshot);
+ this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot);
+ }
+
+ List> nextSnapshots = new ArrayList<>();
+ this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v)));
+ this.nextSnapshotState.update(nextSnapshots);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot);
+ }
+ }
+
+ @SuppressWarnings("BusyWait")
+ @Override
+ public void run(SourceContext ctx) throws Exception {
+ this.ctx = ctx;
+ while (isRunning) {
+ boolean isEmpty;
+ synchronized (ctx.getCheckpointLock()) {
+ if (!isRunning) {
+ return;
+ }
+ try {
+ List splits = scan.plan().splits();
+ isEmpty = splits.isEmpty();
+ splits.forEach(ctx::collect);
+
+ if (emitSnapshotWatermark) {
+ Long watermark = scan.watermark();
+ if (watermark != null) {
+ ctx.emitWatermark(new Watermark(watermark));
+ }
+ }
+ } catch (EndOfScanException esf) {
+ LOG.info("Catching EndOfStreamException, the stream is finished.");
+ return;
+ }
+ }
+
+ if (isEmpty) {
+ Thread.sleep(monitorInterval);
+ }
+ }
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+ NavigableMap nextSnapshots =
+ nextSnapshotPerCheckpoint.headMap(checkpointId, true);
+ OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max();
+ max.ifPresent(scan::notifyCheckpointComplete);
+ nextSnapshots.clear();
+ }
+
+ @Override
+ public void cancel() {
+ // this is to cover the case where cancel() is called before the run()
+ if (ctx != null) {
+ synchronized (ctx.getCheckpointLock()) {
+ isRunning = false;
+ }
+ } else {
+ isRunning = false;
+ }
+ }
+
+ public static DataStream buildSource(
+ StreamExecutionEnvironment env,
+ String name,
+ TypeInformation typeInfo,
+ ReadBuilder readBuilder,
+ long monitorInterval,
+ boolean emitSnapshotWatermark,
+ boolean shuffleBucketWithPartition,
+ BucketMode bucketMode) {
+ SingleOutputStreamOperator singleOutputStreamOperator =
+ env.addSource(
+ new MonitorSource(
+ readBuilder, monitorInterval, emitSnapshotWatermark),
+ name + "-Monitor",
+ new JavaTypeInfo<>(Split.class))
+ .forceNonParallel();
+
+ DataStream sourceDataStream =
+ bucketMode == BUCKET_UNAWARE
+ ? shuffleUnwareBucket(singleOutputStreamOperator)
+ : shuffleNonUnwareBucket(
+ singleOutputStreamOperator, shuffleBucketWithPartition);
+
+ return sourceDataStream.transform(
+ name + "-Reader", typeInfo, new ReadOperator(readBuilder));
+ }
+
+ private static DataStream shuffleUnwareBucket(
+ SingleOutputStreamOperator singleOutputStreamOperator) {
+ return singleOutputStreamOperator.rebalance();
+ }
+
+ private static DataStream shuffleNonUnwareBucket(
+ SingleOutputStreamOperator singleOutputStreamOperator,
+ boolean shuffleBucketWithPartition) {
+ return singleOutputStreamOperator.partitionCustom(
+ (key, numPartitions) -> {
+ if (shuffleBucketWithPartition) {
+ return ChannelComputer.select(key.f0, key.f1, numPartitions);
+ }
+ return ChannelComputer.select(key.f1, numPartitions);
+ },
+ split -> {
+ DataSplit dataSplit = (DataSplit) split;
+ return Tuple2.of(dataSplit.partition(), dataSplit.bucket());
+ });
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java
new file mode 100644
index 000000000000..460fea55ad7a
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.utils;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+/** Utility methods about Flink runtime context to resolve compatibility issues. */
+public class RuntimeContextUtils {
+ public static int getNumberOfParallelSubtasks(RuntimeContext context) {
+ return context.getNumberOfParallelSubtasks();
+ }
+
+ public static int getIndexOfThisSubtask(RuntimeContext context) {
+ return context.getIndexOfThisSubtask();
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java
new file mode 100644
index 000000000000..698900436e8d
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.sink.cdc;
+
+import org.apache.paimon.types.BigIntType;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.RowKind;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.types.VarCharType;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link RichCdcMultiplexRecord} be deserialized by flink KryoSerializer. */
+public class CdcRecordSerializeITCase {
+
+ @Test
+ public void testCdcRecordKryoSerialize() throws IOException {
+ KryoSerializer kr =
+ createFlinkKryoSerializer(RichCdcMultiplexRecord.class);
+ RowType.Builder rowType = RowType.builder();
+ rowType.field("id", new BigIntType());
+ rowType.field("name", new VarCharType());
+ rowType.field("pt", new VarCharType());
+ // this is an unmodifiable list.
+ List fields = rowType.build().getFields();
+ List primaryKeys = Collections.singletonList("id");
+ Map recordData = new HashMap<>();
+ recordData.put("id", "1");
+ recordData.put("name", "HunterXHunter");
+ recordData.put("pt", "2024-06-28");
+ CdcRecord cdcRecord = new CdcRecord(RowKind.INSERT, recordData);
+ RichCdcMultiplexRecord serializeRecord =
+ new RichCdcMultiplexRecord("default", "T", fields, primaryKeys, cdcRecord);
+
+ TestOutputView outputView = new TestOutputView();
+ kr.serialize(serializeRecord, outputView);
+ RichCdcMultiplexRecord deserializeRecord = kr.deserialize(outputView.getInputView());
+ assertThat(deserializeRecord.toRichCdcRecord().toCdcRecord()).isEqualTo(cdcRecord);
+ assertThat(deserializeRecord.databaseName()).isEqualTo("default");
+ assertThat(deserializeRecord.tableName()).isEqualTo("T");
+ assertThat(deserializeRecord.primaryKeys()).isEqualTo(primaryKeys);
+ assertThat(deserializeRecord.fields()).isEqualTo(fields);
+ }
+
+ @Test
+ public void testUnmodifiableListKryoSerialize() throws IOException {
+ KryoSerializer kryoSerializer = createFlinkKryoSerializer(List.class);
+ RowType.Builder rowType = RowType.builder();
+ rowType.field("id", new BigIntType());
+ rowType.field("name", new VarCharType());
+ rowType.field("pt", new VarCharType());
+ // Deserializing an unmodifiable list would be throw
+ // java.lang.UnsupportedOperationException.
+ List fields = rowType.build().getFields();
+
+ TestOutputView outputView = new TestOutputView();
+ kryoSerializer.serialize(fields, outputView);
+ assertThatThrownBy(() -> kryoSerializer.deserialize(outputView.getInputView()))
+ .satisfies(anyCauseMatches(UnsupportedOperationException.class));
+
+ // This `fields` is a modifiable list should be successfully serialized.
+ TestOutputView outputView2 = new TestOutputView();
+ fields = new ArrayList<>(fields);
+ kryoSerializer.serialize(fields, outputView2);
+ List deserializeRecord = kryoSerializer.deserialize(outputView2.getInputView());
+ assertThat(deserializeRecord).isEqualTo(fields);
+ }
+
+ public static KryoSerializer createFlinkKryoSerializer(Class type) {
+ return new KryoSerializer<>(type, new ExecutionConfig());
+ }
+
+ private static final class TestOutputView extends DataOutputStream implements DataOutputView {
+
+ public TestOutputView() {
+ super(new ByteArrayOutputStream(4096));
+ }
+
+ public TestInputView getInputView() {
+ ByteArrayOutputStream baos = (ByteArrayOutputStream) out;
+ return new TestInputView(baos.toByteArray());
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ for (int i = 0; i < numBytes; i++) {
+ write(0);
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ byte[] buffer = new byte[numBytes];
+ source.readFully(buffer);
+ write(buffer);
+ }
+ }
+
+ private static final class TestInputView extends DataInputStream implements DataInputView {
+
+ public TestInputView(byte[] data) {
+ super(new ByteArrayInputStream(data));
+ }
+
+ @Override
+ public void skipBytesToRead(int numBytes) throws IOException {
+ while (numBytes > 0) {
+ int skipped = skipBytes(numBytes);
+ numBytes -= skipped;
+ }
+ }
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/JobInfo.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/JobInfo.java
new file mode 100644
index 000000000000..38e3bfea6965
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/JobInfo.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** The {@link JobInfo} represents the meta information of current job. */
+@PublicEvolving
+public interface JobInfo {
+
+ /**
+ * Get the ID of the job.
+ *
+ * @return the ID of the job
+ */
+ JobID getJobId();
+
+ /**
+ * Get the name of the job.
+ *
+ * @return the name of the job
+ */
+ String getJobName();
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java
new file mode 100644
index 000000000000..21fca4e4c319
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** The default implementation of {@link OpenContext}. */
+@PublicEvolving
+public class DefaultOpenContext implements OpenContext {
+
+ public static final OpenContext INSTANCE = new DefaultOpenContext();
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
new file mode 100644
index 000000000000..4ff5484b3b08
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * The {@link OpenContext} interface provides necessary information required by the {@link
+ * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be
+ * used to add more methods without affecting the signature of {@code RichFunction#open}.
+ */
+@PublicEvolving
+public interface OpenContext {}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/RichFunction.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
new file mode 100644
index 000000000000..ae83fb30f2bd
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * An base interface for all rich user-defined functions. This class defines methods for the life
+ * cycle of the functions, as well as methods to access the context in which the functions are
+ * executed.
+ */
+@Public
+public interface RichFunction extends Function {
+
+ /**
+ * Initialization method for the function. It is called before the actual working methods (like
+ * map or join) and thus suitable for one time setup work. For functions that are
+ * part of an iteration, this method will be invoked at the beginning of each iteration
+ * superstep.
+ *
+ * The configuration object passed to the function can be used for configuration and
+ * initialization. The configuration contains all parameters that were configured on the
+ * function in the program composition.
+ *
+ *
{@code
+ * public class MyFilter extends RichFilterFunction {
+ *
+ * private String searchString;
+ *
+ * public void open(Configuration parameters) {
+ * this.searchString = parameters.getString("foo");
+ * }
+ *
+ * public boolean filter(String value) {
+ * return value.equals(searchString);
+ * }
+ * }
+ * }
+ *
+ * By default, this method does nothing.
+ *
+ * @param parameters The configuration containing the parameters attached to the contract.
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ * @see org.apache.flink.configuration.Configuration
+ * @deprecated This method is deprecated since Flink 1.19. The users are recommended to
+ * implement {@code open(OpenContext openContext)} and implement {@code open(Configuration
+ * parameters)} with an empty body instead. 1. If you implement {@code open(OpenContext
+ * openContext)}, the {@code open(OpenContext openContext)} will be invoked and the {@code
+ * open(Configuration parameters)} won't be invoked. 2. If you don't implement {@code
+ * open(OpenContext openContext)}, the {@code open(Configuration parameters)} will be
+ * invoked in the default implementation of the {@code open(OpenContext openContext)}.
+ * @see
+ * FLIP-344: Remove parameter in RichFunction#open
+ */
+ @Deprecated
+ void open(Configuration parameters) throws Exception;
+
+ /**
+ * Initialization method for the function. It is called before the actual working methods (like
+ * map or join) and thus suitable for one time setup work. For functions that are
+ * part of an iteration, this method will be invoked at the beginning of each iteration
+ * superstep.
+ *
+ *
The openContext object passed to the function can be used for configuration and
+ * initialization. The openContext contains some necessary information that were configured on
+ * the function in the program composition.
+ *
+ *
{@code
+ * public class MyFilter extends RichFilterFunction {
+ *
+ * private String searchString;
+ *
+ * public void open(OpenContext openContext) {
+ * // initialize the value of searchString
+ * }
+ *
+ * public boolean filter(String value) {
+ * return value.equals(searchString);
+ * }
+ * }
+ * }
+ *
+ * By default, this method does nothing.
+ *
+ *
1. If you implement {@code open(OpenContext openContext)}, the {@code open(OpenContext
+ * openContext)} will be invoked and the {@code open(Configuration parameters)} won't be
+ * invoked. 2. If you don't implement {@code open(OpenContext openContext)}, the {@code
+ * open(Configuration parameters)} will be invoked in the default implementation of the {@code
+ * open(OpenContext openContext)}.
+ *
+ * @param openContext The context containing information about the context in which the function
+ * is opened.
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ */
+ @PublicEvolving
+ default void open(OpenContext openContext) throws Exception {
+ open(new Configuration());
+ }
+
+ /**
+ * Tear-down method for the user code. It is called after the last call to the main working
+ * methods (e.g. map or join). For functions that are part of an iteration, this
+ * method will be invoked after each iteration superstep.
+ *
+ *
This method can be used for clean up work.
+ *
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ */
+ void close() throws Exception;
+
+ // ------------------------------------------------------------------------
+ // Runtime context
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the context that contains information about the UDF's runtime, such as the parallelism
+ * of the function, the subtask index of the function, or the name of the task that executes the
+ * function.
+ *
+ *
The RuntimeContext also gives access to the {@link
+ * org.apache.flink.api.common.accumulators.Accumulator}s and the {@link
+ * org.apache.flink.api.common.cache.DistributedCache}.
+ *
+ * @return The UDF's runtime context.
+ */
+ RuntimeContext getRuntimeContext();
+
+ /**
+ * Gets a specialized version of the {@link RuntimeContext}, which has additional information
+ * about the iteration in which the function is executed. This IterationRuntimeContext is only
+ * available if the function is part of an iteration. Otherwise, this method throws an
+ * exception.
+ *
+ * @return The IterationRuntimeContext.
+ * @throws java.lang.IllegalStateException Thrown, if the function is not executed as part of an
+ * iteration.
+ */
+ IterationRuntimeContext getIterationRuntimeContext();
+
+ /**
+ * Sets the function's runtime context. Called by the framework when creating a parallel
+ * instance of the function.
+ *
+ * @param t The runtime context.
+ */
+ void setRuntimeContext(RuntimeContext t);
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java
new file mode 100644
index 000000000000..5e13a9b62329
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.serialization;
+
+import com.esotericsoftware.kryo.Serializer;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInfoFactory;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.util.TernaryBoolean;
+
+import java.io.Serializable;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+
+/**
+ * A config to define the behavior for serializers in Flink job, it manages the registered types and
+ * serializers. The config is created from job configuration and used by Flink to create serializers
+ * for data types.
+ */
+@PublicEvolving
+public interface SerializerConfig extends Serializable {
+ /**
+ * Adds a new Kryo default serializer to the Runtime.
+ *
+ *
Note that the serializer instance must be serializable (as defined by
+ * java.io.Serializable), because it may be distributed to the worker nodes by java
+ * serialization.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializer The serializer to use.
+ */
+ @Internal
+ & Serializable> void addDefaultKryoSerializer(
+ Class> type, T serializer);
+
+ /**
+ * Adds a new Kryo default serializer to the Runtime.
+ *
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializerClass The class of the serializer to use.
+ */
+ @Internal
+ void addDefaultKryoSerializer(Class> type, Class extends Serializer>> serializerClass);
+
+ /**
+ * Registers the given type with a Kryo Serializer.
+ *
+ *
Note that the serializer instance must be serializable (as defined by
+ * java.io.Serializable), because it may be distributed to the worker nodes by java
+ * serialization.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializer The serializer to use.
+ */
+ @Internal
+ & Serializable> void registerTypeWithKryoSerializer(
+ Class> type, T serializer);
+
+ /**
+ * Registers the given Serializer via its class as a serializer for the given type at the
+ * KryoSerializer.
+ *
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializerClass The class of the serializer to use.
+ */
+ @Internal
+ @SuppressWarnings("rawtypes")
+ void registerTypeWithKryoSerializer(Class> type, Class extends Serializer> serializerClass);
+
+ /**
+ * Registers the given type with the serialization stack. If the type is eventually serialized
+ * as a POJO, then the type is registered with the POJO serializer. If the type ends up being
+ * serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
+ * written.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the type to register.
+ */
+ @Internal
+ void registerPojoType(Class> type);
+
+ /**
+ * Registers the given type with the serialization stack. If the type is eventually serialized
+ * as a POJO, then the type is registered with the POJO serializer. If the type ends up being
+ * serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
+ * written.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the type to register.
+ */
+ @Internal
+ void registerKryoType(Class> type);
+
+ /**
+ * Returns the registered types with Kryo Serializers.
+ *
+ * @deprecated The method is deprecated because instance-type Kryo serializer definition based
+ * on {@link ExecutionConfig.SerializableSerializer} is deprecated. Use class-type Kryo
+ * serializers instead.
+ */
+ @Deprecated
+ LinkedHashMap, ExecutionConfig.SerializableSerializer>>
+ getRegisteredTypesWithKryoSerializers();
+
+ /** Returns the registered types with their Kryo Serializer classes. */
+ LinkedHashMap, Class extends Serializer>>>
+ getRegisteredTypesWithKryoSerializerClasses();
+
+ /**
+ * Returns the registered default Kryo Serializers.
+ *
+ * @deprecated The method is deprecated because {@link ExecutionConfig.SerializableSerializer}
+ * is deprecated.
+ */
+ @Deprecated
+ LinkedHashMap, ExecutionConfig.SerializableSerializer>> getDefaultKryoSerializers();
+
+ /** Returns the registered default Kryo Serializer classes. */
+ LinkedHashMap, Class extends Serializer>>> getDefaultKryoSerializerClasses();
+
+ /** Returns the registered Kryo types. */
+ LinkedHashSet> getRegisteredKryoTypes();
+
+ /** Returns the registered POJO types. */
+ LinkedHashSet> getRegisteredPojoTypes();
+
+ /** Returns the registered type info factories. */
+ Map, Class extends TypeInfoFactory>>> getRegisteredTypeInfoFactories();
+
+ /**
+ * Checks whether generic types are supported. Generic types are types that go through Kryo
+ * during serialization.
+ *
+ * Generic types are enabled by default.
+ */
+ boolean hasGenericTypesDisabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setGenericTypes(boolean genericTypes);
+
+ /** Returns whether Kryo is the serializer for POJOs. */
+ boolean isForceKryoEnabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setForceKryo(boolean forceKryo);
+
+ /** Returns whether the Apache Avro is the serializer for POJOs. */
+ boolean isForceAvroEnabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setForceAvro(boolean forceAvro);
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ public void setForceKryoAvro(boolean forceKryoAvro);
+
+ /** Returns whether forces Flink to register Apache Avro classes in Kryo serializer. */
+ TernaryBoolean isForceKryoAvroEnabled();
+
+ /**
+ * Sets all relevant options contained in the {@link ReadableConfig} such as e.g. {@link
+ * PipelineOptions#FORCE_KRYO}.
+ *
+ *
It will change the value of a setting only if a corresponding option was set in the {@code
+ * configuration}. If a key is not present, the current value of a field will remain untouched.
+ *
+ * @param configuration a configuration to read the values from
+ * @param classLoader a class loader to use when loading classes
+ */
+ void configure(ReadableConfig configuration, ClassLoader classLoader);
+
+ SerializerConfig copy();
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java
new file mode 100644
index 000000000000..ae637d6456e7
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.connector.sink2;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobInfo;
+import org.apache.flink.api.common.TaskInfo;
+
+import java.util.OptionalLong;
+
+/**
+ * Common interface which exposes runtime info for creating {@link SinkWriter} and {@link Committer}
+ * objects.
+ */
+@Internal
+public interface InitContext {
+ /**
+ * The first checkpoint id when an application is started and not recovered from a previously
+ * taken checkpoint or savepoint.
+ */
+ long INITIAL_CHECKPOINT_ID = 1;
+
+ /**
+ * Get the id of task where the committer is running.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getSubtaskId() {
+ return getTaskInfo().getIndexOfThisSubtask();
+ }
+
+ /**
+ * Get the number of parallel committer tasks.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getNumberOfParallelSubtasks() {
+ return getTaskInfo().getNumberOfParallelSubtasks();
+ }
+
+ /**
+ * Gets the attempt number of this parallel subtask. First attempt is numbered 0.
+ *
+ * @return Attempt number of the subtask.
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getAttemptNumber() {
+ return getTaskInfo().getAttemptNumber();
+ }
+
+ /**
+ * Returns id of the restored checkpoint, if state was restored from the snapshot of a previous
+ * execution.
+ */
+ OptionalLong getRestoredCheckpointId();
+
+ /**
+ * The ID of the current job. Note that Job ID can change in particular upon manual restart. The
+ * returned ID should NOT be used for any job management tasks.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the job should be
+ * provided uniformly by {@link #getJobInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default JobID getJobId() {
+ return getJobInfo().getJobId();
+ }
+
+ /**
+ * Get the meta information of current job.
+ *
+ * @return the job meta information.
+ */
+ @PublicEvolving
+ JobInfo getJobInfo();
+
+ /**
+ * Get the meta information of current task.
+ *
+ * @return the task meta information.
+ */
+ @PublicEvolving
+ TaskInfo getTaskInfo();
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java
new file mode 100644
index 000000000000..2211ce47bd6f
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.connector.sink2;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import java.util.Optional;
+import java.util.function.Consumer;
+
+/** The interface exposes some runtime info for creating a {@link SinkWriter}. */
+@Public
+public interface WriterInitContext extends org.apache.flink.api.connector.sink2.InitContext {
+ /**
+ * Gets the {@link UserCodeClassLoader} to load classes that are not in system's classpath, but
+ * are part of the jar file of a user job.
+ *
+ * @see UserCodeClassLoader
+ */
+ UserCodeClassLoader getUserCodeClassLoader();
+
+ /**
+ * Returns the mailbox executor that allows to execute {@link Runnable}s inside the task thread
+ * in between record processing.
+ *
+ *
Note that this method should not be used per-record for performance reasons in the same
+ * way as records should not be sent to the external system individually. Rather, implementers
+ * are expected to batch records and only enqueue a single {@link Runnable} per batch to handle
+ * the result.
+ */
+ MailboxExecutor getMailboxExecutor();
+
+ /**
+ * Returns a {@link ProcessingTimeService} that can be used to get the current time and register
+ * timers.
+ */
+ ProcessingTimeService getProcessingTimeService();
+
+ /** @return The metric group this writer belongs to. */
+ SinkWriterMetricGroup metricGroup();
+
+ /** Provides a view on this context as a {@link SerializationSchema.InitializationContext}. */
+ SerializationSchema.InitializationContext asSerializationSchemaInitializationContext();
+
+ /** Returns whether object reuse has been enabled or disabled. */
+ boolean isObjectReuseEnabled();
+
+ /** Creates a serializer for the type of sink's input. */
+ TypeSerializer createInputSerializer();
+
+ /**
+ * Returns a metadata consumer, the {@link SinkWriter} can publish metadata events of type
+ * {@link MetaT} to the consumer.
+ *
+ * It is recommended to use a separate thread pool to publish the metadata because enqueuing
+ * a lot of these messages in the mailbox may lead to a performance decrease. thread, and the
+ * {@link Consumer#accept} method is executed very fast.
+ */
+ default Optional> metadataConsumer() {
+ return Optional.empty();
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
new file mode 100644
index 000000000000..7ddd6a04392a
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+import org.apache.flink.configuration.description.TextElement;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+
+import java.time.Duration;
+
+/** A collection of all configuration options that relate to checkpoints and savepoints. */
+public class CheckpointingOptions {
+
+ // ------------------------------------------------------------------------
+ // general checkpoint options
+ // ------------------------------------------------------------------------
+
+ /**
+ * The checkpoint storage used to store operator state locally within the cluster during
+ * execution.
+ *
+ * The implementation can be specified either via their shortcut name, or via the class name
+ * of a {@code StateBackendFactory}. If a StateBackendFactory class name is specified, the
+ * factory is instantiated (via its zero-argument constructor) and its {@code
+ * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called.
+ *
+ *
Recognized shortcut names are 'hashmap' and 'rocksdb'.
+ *
+ * @deprecated Use {@link StateBackendOptions#STATE_BACKEND}.
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS)
+ @Documentation.ExcludeFromDocumentation("Hidden for deprecated")
+ @Deprecated
+ public static final ConfigOption STATE_BACKEND =
+ ConfigOptions.key("state.backend.type")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("state.backend")
+ .withDescription(
+ Description.builder()
+ .text("The state backend to be used to store state.")
+ .linebreak()
+ .text(
+ "The implementation can be specified either via their shortcut "
+ + " name, or via the class name of a %s. "
+ + "If a factory is specified it is instantiated via its "
+ + "zero argument constructor and its %s "
+ + "method is called.",
+ TextElement.code("StateBackendFactory"),
+ TextElement.code(
+ "StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text("Recognized shortcut names are 'hashmap' and 'rocksdb'.")
+ .build());
+
+ /**
+ * The checkpoint storage used to checkpoint state for recovery.
+ *
+ * The implementation can be specified either via their shortcut name, or via the class name
+ * of a {@code CheckpointStorageFactory}. If a CheckpointStorageFactory class name is specified,
+ * the factory is instantiated (via its zero-argument constructor) and its {@code
+ * CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called.
+ *
+ *
Recognized shortcut names are 'jobmanager' and 'filesystem'.
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
+ public static final ConfigOption CHECKPOINT_STORAGE =
+ ConfigOptions.key("state.checkpoint-storage")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The checkpoint storage implementation to be used to checkpoint state.")
+ .linebreak()
+ .text(
+ "The implementation can be specified either via their shortcut "
+ + " name, or via the class name of a %s. "
+ + "If a factory is specified it is instantiated via its "
+ + "zero argument constructor and its %s "
+ + " method is called.",
+ TextElement.code("CheckpointStorageFactory"),
+ TextElement.code(
+ "CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text(
+ "Recognized shortcut names are 'jobmanager' and 'filesystem'.")
+ .build());
+
+ /** The maximum number of completed checkpoints to retain. */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption MAX_RETAINED_CHECKPOINTS =
+ ConfigOptions.key("state.checkpoints.num-retained")
+ .intType()
+ .defaultValue(1)
+ .withDescription("The maximum number of completed checkpoints to retain.");
+
+ /** @deprecated Checkpoints are always asynchronous. */
+ @Deprecated
+ public static final ConfigOption ASYNC_SNAPSHOTS =
+ ConfigOptions.key("state.backend.async")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription("Deprecated option. All state snapshots are asynchronous.");
+
+ /**
+ * Option whether the state backend should create incremental checkpoints, if possible. For an
+ * incremental checkpoint, only a diff from the previous checkpoint is stored, rather than the
+ * complete checkpoint state.
+ *
+ * Once enabled, the state size shown in web UI or fetched from rest API only represents the
+ * delta checkpoint size instead of full checkpoint size.
+ *
+ *
Some state backends may not support incremental checkpoints and ignore this option.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption INCREMENTAL_CHECKPOINTS =
+ ConfigOptions.key("state.backend.incremental")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Option whether the state backend should create incremental checkpoints, if possible. For"
+ + " an incremental checkpoint, only a diff from the previous checkpoint is stored, rather than the"
+ + " complete checkpoint state. Once enabled, the state size shown in web UI or fetched from rest API"
+ + " only represents the delta checkpoint size instead of full checkpoint size."
+ + " Some state backends may not support incremental checkpoints and ignore this option.");
+
+ /**
+ * This option configures local recovery for this state backend. By default, local recovery is
+ * deactivated.
+ *
+ * Local recovery currently only covers keyed state backends (including both the
+ * EmbeddedRocksDBStateBackend and the HashMapStateBackend).
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption LOCAL_RECOVERY =
+ ConfigOptions.key("state.backend.local-recovery")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "This option configures local recovery for this state backend. By default, local recovery is "
+ + "deactivated. Local recovery currently only covers keyed state backends "
+ + "(including both the EmbeddedRocksDBStateBackend and the HashMapStateBackend).");
+
+ /**
+ * The config parameter defining the root directories for storing file-based state for local
+ * recovery.
+ *
+ * Local recovery currently only covers keyed state backends. Currently, MemoryStateBackend
+ * does not support local recovery and ignore this option.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS =
+ ConfigOptions.key("taskmanager.state.local.root-dirs")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The config parameter defining the root directories for storing file-based "
+ + "state for local recovery. Local recovery currently only covers keyed "
+ + "state backends. If not configured it will default to /localState. "
+ + "The can be configured via %s",
+ TextElement.code(
+ ClusterOptions
+ .TASK_MANAGER_PROCESS_WORKING_DIR_BASE
+ .key()))
+ .build());
+
+ // ------------------------------------------------------------------------
+ // Options specific to the file-system-based state backends
+ // ------------------------------------------------------------------------
+
+ /**
+ * The default directory for savepoints. Used by the state backends that write savepoints to
+ * file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 3)
+ public static final ConfigOption SAVEPOINT_DIRECTORY =
+ ConfigOptions.key("state.savepoints.dir")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("savepoints.state.backend.fs.dir")
+ .withDescription(
+ "The default directory for savepoints. Used by the state backends that write savepoints to"
+ + " file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).");
+
+ /**
+ * The default directory used for storing the data files and meta data of checkpoints in a Flink
+ * supported filesystem. The storage path must be accessible from all participating
+ * processes/nodes(i.e. all TaskManagers and JobManagers).
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
+ public static final ConfigOption CHECKPOINTS_DIRECTORY =
+ ConfigOptions.key("state.checkpoints.dir")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("state.backend.fs.checkpointdir")
+ .withDescription(
+ "The default directory used for storing the data files and meta data of checkpoints "
+ + "in a Flink supported filesystem. The storage path must be accessible from all participating processes/nodes"
+ + "(i.e. all TaskManagers and JobManagers).");
+
+ /**
+ * The minimum size of state data files. All state chunks smaller than that are stored inline in
+ * the root checkpoint metadata file.
+ */
+ @Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS)
+ public static final ConfigOption FS_SMALL_FILE_THRESHOLD =
+ ConfigOptions.key("state.storage.fs.memory-threshold")
+ .memoryType()
+ .defaultValue(MemorySize.parse("20kb"))
+ .withDescription(
+ "The minimum size of state data files. All state chunks smaller than that are stored"
+ + " inline in the root checkpoint metadata file. The max memory threshold for this configuration is 1MB.")
+ .withDeprecatedKeys("state.backend.fs.memory-threshold");
+
+ /**
+ * The default size of the write buffer for the checkpoint streams that write to file systems.
+ */
+ @Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS)
+ public static final ConfigOption FS_WRITE_BUFFER_SIZE =
+ ConfigOptions.key("state.storage.fs.write-buffer-size")
+ .intType()
+ .defaultValue(4 * 1024)
+ .withDescription(
+ String.format(
+ "The default size of the write buffer for the checkpoint streams that write to file systems. "
+ + "The actual write buffer size is determined to be the maximum of the value of this option and option '%s'.",
+ FS_SMALL_FILE_THRESHOLD.key()))
+ .withDeprecatedKeys("state.backend.fs.write-buffer-size");
+
+ public static final ConfigOption CHECKPOINTING_INTERVAL =
+ ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+
+ public static final ConfigOption ENABLE_UNALIGNED =
+ ExecutionCheckpointingOptions.ENABLE_UNALIGNED;
+
+ public static final ConfigOption CHECKPOINTING_CONSISTENCY_MODE =
+ ExecutionCheckpointingOptions.CHECKPOINTING_MODE;
+
+ public static final ConfigOption MAX_CONCURRENT_CHECKPOINTS =
+ ExecutionCheckpointingOptions.MAX_CONCURRENT_CHECKPOINTS;
+
+ public static final ConfigOption CHECKPOINTING_TIMEOUT =
+ ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT;
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java
new file mode 100644
index 000000000000..2193c2a6aab9
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The {@link ConfigOption configuration options} used when restoring state from a savepoint or a
+ * checkpoint.
+ */
+@PublicEvolving
+public class StateRecoveryOptions {
+ /** The path to a savepoint that will be used to bootstrap the pipeline's state. */
+ public static final ConfigOption SAVEPOINT_PATH =
+ key("execution.state-recovery.path")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("execution.savepoint.path")
+ .withDescription(
+ "Path to a savepoint to restore the job from (for example hdfs:///flink/savepoint-1537).");
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
new file mode 100644
index 000000000000..fc7eb0d48356
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.sink.v2;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+
+import java.io.IOException;
+
+/**
+ * A special sink that ignores all elements.
+ *
+ * @param The type of elements received by the sink.
+ */
+@PublicEvolving
+public class DiscardingSink implements Sink, SupportsConcurrentExecutionAttempts {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public SinkWriter createWriter(InitContext context) throws IOException {
+ return new DiscardingElementWriter();
+ }
+
+ private class DiscardingElementWriter implements SinkWriter {
+
+ @Override
+ public void write(IN element, Context context) throws IOException, InterruptedException {
+ // discard it.
+ }
+
+ @Override
+ public void flush(boolean endOfInput) throws IOException, InterruptedException {
+ // this writer has no pending data.
+ }
+
+ @Override
+ public void close() throws Exception {
+ // do nothing.
+ }
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java
new file mode 100644
index 000000000000..b38e32559c36
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.source.operator;
+
+import org.apache.paimon.flink.utils.JavaTypeInfo;
+import org.apache.paimon.table.BucketMode;
+import org.apache.paimon.table.sink.ChannelComputer;
+import org.apache.paimon.table.source.DataSplit;
+import org.apache.paimon.table.source.EndOfScanException;
+import org.apache.paimon.table.source.ReadBuilder;
+import org.apache.paimon.table.source.Split;
+import org.apache.paimon.table.source.StreamTableScan;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+
+import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE;
+
+/**
+ * This is the single (non-parallel) monitoring task, it is responsible for:
+ *
+ *
+ * - Monitoring snapshots of the Paimon table.
+ *
- Creating the {@link Split splits} corresponding to the incremental files
+ *
- Assigning them to downstream tasks for further processing.
+ *
+ *
+ * The splits to be read are forwarded to the downstream {@link ReadOperator} which can have
+ * parallelism greater than one.
+ *
+ *
Currently, there are two features that rely on this monitor:
+ *
+ *
+ * - Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all
+ * data in a snapshot is consumed within each checkpoint.
+ *
- Snapshot-watermark: when there is no watermark definition, the default Paimon table will
+ * pass the watermark recorded in the snapshot.
+ *
+ */
+public class MonitorSource extends RichSourceFunction
+ implements CheckpointedFunction, CheckpointListener {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class);
+
+ private final ReadBuilder readBuilder;
+ private final long monitorInterval;
+ private final boolean emitSnapshotWatermark;
+
+ private volatile boolean isRunning = true;
+
+ private transient StreamTableScan scan;
+ private transient SourceContext ctx;
+
+ private transient ListState checkpointState;
+ private transient ListState> nextSnapshotState;
+ private transient TreeMap nextSnapshotPerCheckpoint;
+
+ public MonitorSource(
+ ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) {
+ this.readBuilder = readBuilder;
+ this.monitorInterval = monitorInterval;
+ this.emitSnapshotWatermark = emitSnapshotWatermark;
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ this.scan = readBuilder.newStreamScan();
+
+ this.checkpointState =
+ context.getOperatorStateStore()
+ .getListState(
+ new ListStateDescriptor<>(
+ "next-snapshot", LongSerializer.INSTANCE));
+
+ @SuppressWarnings("unchecked")
+ final Class> typedTuple =
+ (Class>) (Class>) Tuple2.class;
+ this.nextSnapshotState =
+ context.getOperatorStateStore()
+ .getListState(
+ new ListStateDescriptor<>(
+ "next-snapshot-per-checkpoint",
+ new TupleSerializer<>(
+ typedTuple,
+ new TypeSerializer[] {
+ LongSerializer.INSTANCE, LongSerializer.INSTANCE
+ })));
+
+ this.nextSnapshotPerCheckpoint = new TreeMap<>();
+
+ if (context.isRestored()) {
+ LOG.info("Restoring state for the {}.", getClass().getSimpleName());
+
+ List retrievedStates = new ArrayList<>();
+ for (Long entry : this.checkpointState.get()) {
+ retrievedStates.add(entry);
+ }
+
+ // given that the parallelism of the function is 1, we can only have 1 retrieved items.
+ Preconditions.checkArgument(
+ retrievedStates.size() <= 1,
+ getClass().getSimpleName() + " retrieved invalid state.");
+
+ if (retrievedStates.size() == 1) {
+ this.scan.restore(retrievedStates.get(0));
+ }
+
+ for (Tuple2 tuple2 : nextSnapshotState.get()) {
+ nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1);
+ }
+ } else {
+ LOG.info("No state to restore for the {}.", getClass().getSimpleName());
+ }
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
+ this.checkpointState.clear();
+ Long nextSnapshot = this.scan.checkpoint();
+ if (nextSnapshot != null) {
+ this.checkpointState.add(nextSnapshot);
+ this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot);
+ }
+
+ List> nextSnapshots = new ArrayList<>();
+ this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v)));
+ this.nextSnapshotState.update(nextSnapshots);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot);
+ }
+ }
+
+ @SuppressWarnings("BusyWait")
+ @Override
+ public void run(SourceContext ctx) throws Exception {
+ this.ctx = ctx;
+ while (isRunning) {
+ boolean isEmpty;
+ synchronized (ctx.getCheckpointLock()) {
+ if (!isRunning) {
+ return;
+ }
+ try {
+ List splits = scan.plan().splits();
+ isEmpty = splits.isEmpty();
+ splits.forEach(ctx::collect);
+
+ if (emitSnapshotWatermark) {
+ Long watermark = scan.watermark();
+ if (watermark != null) {
+ ctx.emitWatermark(new Watermark(watermark));
+ }
+ }
+ } catch (EndOfScanException esf) {
+ LOG.info("Catching EndOfStreamException, the stream is finished.");
+ return;
+ }
+ }
+
+ if (isEmpty) {
+ Thread.sleep(monitorInterval);
+ }
+ }
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+ NavigableMap nextSnapshots =
+ nextSnapshotPerCheckpoint.headMap(checkpointId, true);
+ OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max();
+ max.ifPresent(scan::notifyCheckpointComplete);
+ nextSnapshots.clear();
+ }
+
+ @Override
+ public void cancel() {
+ // this is to cover the case where cancel() is called before the run()
+ if (ctx != null) {
+ synchronized (ctx.getCheckpointLock()) {
+ isRunning = false;
+ }
+ } else {
+ isRunning = false;
+ }
+ }
+
+ public static DataStream buildSource(
+ StreamExecutionEnvironment env,
+ String name,
+ TypeInformation typeInfo,
+ ReadBuilder readBuilder,
+ long monitorInterval,
+ boolean emitSnapshotWatermark,
+ boolean shuffleBucketWithPartition,
+ BucketMode bucketMode) {
+ SingleOutputStreamOperator singleOutputStreamOperator =
+ env.addSource(
+ new MonitorSource(
+ readBuilder, monitorInterval, emitSnapshotWatermark),
+ name + "-Monitor",
+ new JavaTypeInfo<>(Split.class))
+ .forceNonParallel();
+
+ DataStream sourceDataStream =
+ bucketMode == BUCKET_UNAWARE
+ ? shuffleUnwareBucket(singleOutputStreamOperator)
+ : shuffleNonUnwareBucket(
+ singleOutputStreamOperator, shuffleBucketWithPartition);
+
+ return sourceDataStream.transform(
+ name + "-Reader", typeInfo, new ReadOperator(readBuilder));
+ }
+
+ private static DataStream shuffleUnwareBucket(
+ SingleOutputStreamOperator singleOutputStreamOperator) {
+ return singleOutputStreamOperator.rebalance();
+ }
+
+ private static DataStream shuffleNonUnwareBucket(
+ SingleOutputStreamOperator singleOutputStreamOperator,
+ boolean shuffleBucketWithPartition) {
+ return singleOutputStreamOperator.partitionCustom(
+ (key, numPartitions) -> {
+ if (shuffleBucketWithPartition) {
+ return ChannelComputer.select(key.f0, key.f1, numPartitions);
+ }
+ return ChannelComputer.select(key.f1, numPartitions);
+ },
+ split -> {
+ DataSplit dataSplit = (DataSplit) split;
+ return Tuple2.of(dataSplit.partition(), dataSplit.bucket());
+ });
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java
new file mode 100644
index 000000000000..460fea55ad7a
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.utils;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+/** Utility methods about Flink runtime context to resolve compatibility issues. */
+public class RuntimeContextUtils {
+ public static int getNumberOfParallelSubtasks(RuntimeContext context) {
+ return context.getNumberOfParallelSubtasks();
+ }
+
+ public static int getIndexOfThisSubtask(RuntimeContext context) {
+ return context.getIndexOfThisSubtask();
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.17/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java b/paimon-flink/paimon-flink-1.17/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java
new file mode 100644
index 000000000000..698900436e8d
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.17/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.sink.cdc;
+
+import org.apache.paimon.types.BigIntType;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.RowKind;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.types.VarCharType;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link RichCdcMultiplexRecord} be deserialized by flink KryoSerializer. */
+public class CdcRecordSerializeITCase {
+
+ @Test
+ public void testCdcRecordKryoSerialize() throws IOException {
+ KryoSerializer kr =
+ createFlinkKryoSerializer(RichCdcMultiplexRecord.class);
+ RowType.Builder rowType = RowType.builder();
+ rowType.field("id", new BigIntType());
+ rowType.field("name", new VarCharType());
+ rowType.field("pt", new VarCharType());
+ // this is an unmodifiable list.
+ List fields = rowType.build().getFields();
+ List primaryKeys = Collections.singletonList("id");
+ Map recordData = new HashMap<>();
+ recordData.put("id", "1");
+ recordData.put("name", "HunterXHunter");
+ recordData.put("pt", "2024-06-28");
+ CdcRecord cdcRecord = new CdcRecord(RowKind.INSERT, recordData);
+ RichCdcMultiplexRecord serializeRecord =
+ new RichCdcMultiplexRecord("default", "T", fields, primaryKeys, cdcRecord);
+
+ TestOutputView outputView = new TestOutputView();
+ kr.serialize(serializeRecord, outputView);
+ RichCdcMultiplexRecord deserializeRecord = kr.deserialize(outputView.getInputView());
+ assertThat(deserializeRecord.toRichCdcRecord().toCdcRecord()).isEqualTo(cdcRecord);
+ assertThat(deserializeRecord.databaseName()).isEqualTo("default");
+ assertThat(deserializeRecord.tableName()).isEqualTo("T");
+ assertThat(deserializeRecord.primaryKeys()).isEqualTo(primaryKeys);
+ assertThat(deserializeRecord.fields()).isEqualTo(fields);
+ }
+
+ @Test
+ public void testUnmodifiableListKryoSerialize() throws IOException {
+ KryoSerializer kryoSerializer = createFlinkKryoSerializer(List.class);
+ RowType.Builder rowType = RowType.builder();
+ rowType.field("id", new BigIntType());
+ rowType.field("name", new VarCharType());
+ rowType.field("pt", new VarCharType());
+ // Deserializing an unmodifiable list would be throw
+ // java.lang.UnsupportedOperationException.
+ List fields = rowType.build().getFields();
+
+ TestOutputView outputView = new TestOutputView();
+ kryoSerializer.serialize(fields, outputView);
+ assertThatThrownBy(() -> kryoSerializer.deserialize(outputView.getInputView()))
+ .satisfies(anyCauseMatches(UnsupportedOperationException.class));
+
+ // This `fields` is a modifiable list should be successfully serialized.
+ TestOutputView outputView2 = new TestOutputView();
+ fields = new ArrayList<>(fields);
+ kryoSerializer.serialize(fields, outputView2);
+ List deserializeRecord = kryoSerializer.deserialize(outputView2.getInputView());
+ assertThat(deserializeRecord).isEqualTo(fields);
+ }
+
+ public static KryoSerializer createFlinkKryoSerializer(Class type) {
+ return new KryoSerializer<>(type, new ExecutionConfig());
+ }
+
+ private static final class TestOutputView extends DataOutputStream implements DataOutputView {
+
+ public TestOutputView() {
+ super(new ByteArrayOutputStream(4096));
+ }
+
+ public TestInputView getInputView() {
+ ByteArrayOutputStream baos = (ByteArrayOutputStream) out;
+ return new TestInputView(baos.toByteArray());
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ for (int i = 0; i < numBytes; i++) {
+ write(0);
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ byte[] buffer = new byte[numBytes];
+ source.readFully(buffer);
+ write(buffer);
+ }
+ }
+
+ private static final class TestInputView extends DataInputStream implements DataInputView {
+
+ public TestInputView(byte[] data) {
+ super(new ByteArrayInputStream(data));
+ }
+
+ @Override
+ public void skipBytesToRead(int numBytes) throws IOException {
+ while (numBytes > 0) {
+ int skipped = skipBytes(numBytes);
+ numBytes -= skipped;
+ }
+ }
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/JobInfo.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/JobInfo.java
new file mode 100644
index 000000000000..38e3bfea6965
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/JobInfo.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** The {@link JobInfo} represents the meta information of current job. */
+@PublicEvolving
+public interface JobInfo {
+
+ /**
+ * Get the ID of the job.
+ *
+ * @return the ID of the job
+ */
+ JobID getJobId();
+
+ /**
+ * Get the name of the job.
+ *
+ * @return the name of the job
+ */
+ String getJobName();
+}
diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java
new file mode 100644
index 000000000000..21fca4e4c319
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** The default implementation of {@link OpenContext}. */
+@PublicEvolving
+public class DefaultOpenContext implements OpenContext {
+
+ public static final OpenContext INSTANCE = new DefaultOpenContext();
+}
diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
new file mode 100644
index 000000000000..4ff5484b3b08
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * The {@link OpenContext} interface provides necessary information required by the {@link
+ * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be
+ * used to add more methods without affecting the signature of {@code RichFunction#open}.
+ */
+@PublicEvolving
+public interface OpenContext {}
diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/RichFunction.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
new file mode 100644
index 000000000000..ae83fb30f2bd
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.functions;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * An base interface for all rich user-defined functions. This class defines methods for the life
+ * cycle of the functions, as well as methods to access the context in which the functions are
+ * executed.
+ */
+@Public
+public interface RichFunction extends Function {
+
+ /**
+ * Initialization method for the function. It is called before the actual working methods (like
+ * map or join) and thus suitable for one time setup work. For functions that are
+ * part of an iteration, this method will be invoked at the beginning of each iteration
+ * superstep.
+ *
+ * The configuration object passed to the function can be used for configuration and
+ * initialization. The configuration contains all parameters that were configured on the
+ * function in the program composition.
+ *
+ *
{@code
+ * public class MyFilter extends RichFilterFunction {
+ *
+ * private String searchString;
+ *
+ * public void open(Configuration parameters) {
+ * this.searchString = parameters.getString("foo");
+ * }
+ *
+ * public boolean filter(String value) {
+ * return value.equals(searchString);
+ * }
+ * }
+ * }
+ *
+ * By default, this method does nothing.
+ *
+ * @param parameters The configuration containing the parameters attached to the contract.
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ * @see org.apache.flink.configuration.Configuration
+ * @deprecated This method is deprecated since Flink 1.19. The users are recommended to
+ * implement {@code open(OpenContext openContext)} and implement {@code open(Configuration
+ * parameters)} with an empty body instead. 1. If you implement {@code open(OpenContext
+ * openContext)}, the {@code open(OpenContext openContext)} will be invoked and the {@code
+ * open(Configuration parameters)} won't be invoked. 2. If you don't implement {@code
+ * open(OpenContext openContext)}, the {@code open(Configuration parameters)} will be
+ * invoked in the default implementation of the {@code open(OpenContext openContext)}.
+ * @see
+ * FLIP-344: Remove parameter in RichFunction#open
+ */
+ @Deprecated
+ void open(Configuration parameters) throws Exception;
+
+ /**
+ * Initialization method for the function. It is called before the actual working methods (like
+ * map or join) and thus suitable for one time setup work. For functions that are
+ * part of an iteration, this method will be invoked at the beginning of each iteration
+ * superstep.
+ *
+ *
The openContext object passed to the function can be used for configuration and
+ * initialization. The openContext contains some necessary information that were configured on
+ * the function in the program composition.
+ *
+ *
{@code
+ * public class MyFilter extends RichFilterFunction {
+ *
+ * private String searchString;
+ *
+ * public void open(OpenContext openContext) {
+ * // initialize the value of searchString
+ * }
+ *
+ * public boolean filter(String value) {
+ * return value.equals(searchString);
+ * }
+ * }
+ * }
+ *
+ * By default, this method does nothing.
+ *
+ *
1. If you implement {@code open(OpenContext openContext)}, the {@code open(OpenContext
+ * openContext)} will be invoked and the {@code open(Configuration parameters)} won't be
+ * invoked. 2. If you don't implement {@code open(OpenContext openContext)}, the {@code
+ * open(Configuration parameters)} will be invoked in the default implementation of the {@code
+ * open(OpenContext openContext)}.
+ *
+ * @param openContext The context containing information about the context in which the function
+ * is opened.
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ */
+ @PublicEvolving
+ default void open(OpenContext openContext) throws Exception {
+ open(new Configuration());
+ }
+
+ /**
+ * Tear-down method for the user code. It is called after the last call to the main working
+ * methods (e.g. map or join). For functions that are part of an iteration, this
+ * method will be invoked after each iteration superstep.
+ *
+ *
This method can be used for clean up work.
+ *
+ * @throws Exception Implementations may forward exceptions, which are caught by the runtime.
+ * When the runtime catches an exception, it aborts the task and lets the fail-over logic
+ * decide whether to retry the task execution.
+ */
+ void close() throws Exception;
+
+ // ------------------------------------------------------------------------
+ // Runtime context
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the context that contains information about the UDF's runtime, such as the parallelism
+ * of the function, the subtask index of the function, or the name of the task that executes the
+ * function.
+ *
+ *
The RuntimeContext also gives access to the {@link
+ * org.apache.flink.api.common.accumulators.Accumulator}s and the {@link
+ * org.apache.flink.api.common.cache.DistributedCache}.
+ *
+ * @return The UDF's runtime context.
+ */
+ RuntimeContext getRuntimeContext();
+
+ /**
+ * Gets a specialized version of the {@link RuntimeContext}, which has additional information
+ * about the iteration in which the function is executed. This IterationRuntimeContext is only
+ * available if the function is part of an iteration. Otherwise, this method throws an
+ * exception.
+ *
+ * @return The IterationRuntimeContext.
+ * @throws java.lang.IllegalStateException Thrown, if the function is not executed as part of an
+ * iteration.
+ */
+ IterationRuntimeContext getIterationRuntimeContext();
+
+ /**
+ * Sets the function's runtime context. Called by the framework when creating a parallel
+ * instance of the function.
+ *
+ * @param t The runtime context.
+ */
+ void setRuntimeContext(RuntimeContext t);
+}
diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java
new file mode 100644
index 000000000000..5e13a9b62329
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.serialization;
+
+import com.esotericsoftware.kryo.Serializer;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInfoFactory;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.util.TernaryBoolean;
+
+import java.io.Serializable;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+
+/**
+ * A config to define the behavior for serializers in Flink job, it manages the registered types and
+ * serializers. The config is created from job configuration and used by Flink to create serializers
+ * for data types.
+ */
+@PublicEvolving
+public interface SerializerConfig extends Serializable {
+ /**
+ * Adds a new Kryo default serializer to the Runtime.
+ *
+ *
Note that the serializer instance must be serializable (as defined by
+ * java.io.Serializable), because it may be distributed to the worker nodes by java
+ * serialization.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializer The serializer to use.
+ */
+ @Internal
+ & Serializable> void addDefaultKryoSerializer(
+ Class> type, T serializer);
+
+ /**
+ * Adds a new Kryo default serializer to the Runtime.
+ *
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializerClass The class of the serializer to use.
+ */
+ @Internal
+ void addDefaultKryoSerializer(Class> type, Class extends Serializer>> serializerClass);
+
+ /**
+ * Registers the given type with a Kryo Serializer.
+ *
+ *
Note that the serializer instance must be serializable (as defined by
+ * java.io.Serializable), because it may be distributed to the worker nodes by java
+ * serialization.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializer The serializer to use.
+ */
+ @Internal
+ & Serializable> void registerTypeWithKryoSerializer(
+ Class> type, T serializer);
+
+ /**
+ * Registers the given Serializer via its class as a serializer for the given type at the
+ * KryoSerializer.
+ *
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the types serialized with the given serializer.
+ * @param serializerClass The class of the serializer to use.
+ */
+ @Internal
+ @SuppressWarnings("rawtypes")
+ void registerTypeWithKryoSerializer(Class> type, Class extends Serializer> serializerClass);
+
+ /**
+ * Registers the given type with the serialization stack. If the type is eventually serialized
+ * as a POJO, then the type is registered with the POJO serializer. If the type ends up being
+ * serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
+ * written.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the type to register.
+ */
+ @Internal
+ void registerPojoType(Class> type);
+
+ /**
+ * Registers the given type with the serialization stack. If the type is eventually serialized
+ * as a POJO, then the type is registered with the POJO serializer. If the type ends up being
+ * serialized with Kryo, then it will be registered at Kryo to make sure that only tags are
+ * written.
+ *
+ *
The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ *
+ * @param type The class of the type to register.
+ */
+ @Internal
+ void registerKryoType(Class> type);
+
+ /**
+ * Returns the registered types with Kryo Serializers.
+ *
+ * @deprecated The method is deprecated because instance-type Kryo serializer definition based
+ * on {@link ExecutionConfig.SerializableSerializer} is deprecated. Use class-type Kryo
+ * serializers instead.
+ */
+ @Deprecated
+ LinkedHashMap, ExecutionConfig.SerializableSerializer>>
+ getRegisteredTypesWithKryoSerializers();
+
+ /** Returns the registered types with their Kryo Serializer classes. */
+ LinkedHashMap, Class extends Serializer>>>
+ getRegisteredTypesWithKryoSerializerClasses();
+
+ /**
+ * Returns the registered default Kryo Serializers.
+ *
+ * @deprecated The method is deprecated because {@link ExecutionConfig.SerializableSerializer}
+ * is deprecated.
+ */
+ @Deprecated
+ LinkedHashMap, ExecutionConfig.SerializableSerializer>> getDefaultKryoSerializers();
+
+ /** Returns the registered default Kryo Serializer classes. */
+ LinkedHashMap, Class extends Serializer>>> getDefaultKryoSerializerClasses();
+
+ /** Returns the registered Kryo types. */
+ LinkedHashSet> getRegisteredKryoTypes();
+
+ /** Returns the registered POJO types. */
+ LinkedHashSet> getRegisteredPojoTypes();
+
+ /** Returns the registered type info factories. */
+ Map, Class extends TypeInfoFactory>>> getRegisteredTypeInfoFactories();
+
+ /**
+ * Checks whether generic types are supported. Generic types are types that go through Kryo
+ * during serialization.
+ *
+ * Generic types are enabled by default.
+ */
+ boolean hasGenericTypesDisabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setGenericTypes(boolean genericTypes);
+
+ /** Returns whether Kryo is the serializer for POJOs. */
+ boolean isForceKryoEnabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setForceKryo(boolean forceKryo);
+
+ /** Returns whether the Apache Avro is the serializer for POJOs. */
+ boolean isForceAvroEnabled();
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ void setForceAvro(boolean forceAvro);
+
+ /**
+ * The method will be converted to private in the next Flink major version after removing its
+ * deprecated caller methods.
+ */
+ @Internal
+ public void setForceKryoAvro(boolean forceKryoAvro);
+
+ /** Returns whether forces Flink to register Apache Avro classes in Kryo serializer. */
+ TernaryBoolean isForceKryoAvroEnabled();
+
+ /**
+ * Sets all relevant options contained in the {@link ReadableConfig} such as e.g. {@link
+ * PipelineOptions#FORCE_KRYO}.
+ *
+ *
It will change the value of a setting only if a corresponding option was set in the {@code
+ * configuration}. If a key is not present, the current value of a field will remain untouched.
+ *
+ * @param configuration a configuration to read the values from
+ * @param classLoader a class loader to use when loading classes
+ */
+ void configure(ReadableConfig configuration, ClassLoader classLoader);
+
+ SerializerConfig copy();
+}
diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java
new file mode 100644
index 000000000000..ae637d6456e7
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.connector.sink2;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobInfo;
+import org.apache.flink.api.common.TaskInfo;
+
+import java.util.OptionalLong;
+
+/**
+ * Common interface which exposes runtime info for creating {@link SinkWriter} and {@link Committer}
+ * objects.
+ */
+@Internal
+public interface InitContext {
+ /**
+ * The first checkpoint id when an application is started and not recovered from a previously
+ * taken checkpoint or savepoint.
+ */
+ long INITIAL_CHECKPOINT_ID = 1;
+
+ /**
+ * Get the id of task where the committer is running.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getSubtaskId() {
+ return getTaskInfo().getIndexOfThisSubtask();
+ }
+
+ /**
+ * Get the number of parallel committer tasks.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getNumberOfParallelSubtasks() {
+ return getTaskInfo().getNumberOfParallelSubtasks();
+ }
+
+ /**
+ * Gets the attempt number of this parallel subtask. First attempt is numbered 0.
+ *
+ * @return Attempt number of the subtask.
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the task should be
+ * provided uniformly by {@link #getTaskInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default int getAttemptNumber() {
+ return getTaskInfo().getAttemptNumber();
+ }
+
+ /**
+ * Returns id of the restored checkpoint, if state was restored from the snapshot of a previous
+ * execution.
+ */
+ OptionalLong getRestoredCheckpointId();
+
+ /**
+ * The ID of the current job. Note that Job ID can change in particular upon manual restart. The
+ * returned ID should NOT be used for any job management tasks.
+ *
+ * @deprecated This method is deprecated since Flink 1.19. All metadata about the job should be
+ * provided uniformly by {@link #getJobInfo()}.
+ * @see
+ * FLIP-382: Unify the Provision of Diverse Metadata for Context-like APIs
+ */
+ @Deprecated
+ default JobID getJobId() {
+ return getJobInfo().getJobId();
+ }
+
+ /**
+ * Get the meta information of current job.
+ *
+ * @return the job meta information.
+ */
+ @PublicEvolving
+ JobInfo getJobInfo();
+
+ /**
+ * Get the meta information of current task.
+ *
+ * @return the task meta information.
+ */
+ @PublicEvolving
+ TaskInfo getTaskInfo();
+}
diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java
new file mode 100644
index 000000000000..2211ce47bd6f
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.connector.sink2;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import java.util.Optional;
+import java.util.function.Consumer;
+
+/** The interface exposes some runtime info for creating a {@link SinkWriter}. */
+@Public
+public interface WriterInitContext extends org.apache.flink.api.connector.sink2.InitContext {
+ /**
+ * Gets the {@link UserCodeClassLoader} to load classes that are not in system's classpath, but
+ * are part of the jar file of a user job.
+ *
+ * @see UserCodeClassLoader
+ */
+ UserCodeClassLoader getUserCodeClassLoader();
+
+ /**
+ * Returns the mailbox executor that allows to execute {@link Runnable}s inside the task thread
+ * in between record processing.
+ *
+ *
Note that this method should not be used per-record for performance reasons in the same
+ * way as records should not be sent to the external system individually. Rather, implementers
+ * are expected to batch records and only enqueue a single {@link Runnable} per batch to handle
+ * the result.
+ */
+ MailboxExecutor getMailboxExecutor();
+
+ /**
+ * Returns a {@link ProcessingTimeService} that can be used to get the current time and register
+ * timers.
+ */
+ ProcessingTimeService getProcessingTimeService();
+
+ /** @return The metric group this writer belongs to. */
+ SinkWriterMetricGroup metricGroup();
+
+ /** Provides a view on this context as a {@link SerializationSchema.InitializationContext}. */
+ SerializationSchema.InitializationContext asSerializationSchemaInitializationContext();
+
+ /** Returns whether object reuse has been enabled or disabled. */
+ boolean isObjectReuseEnabled();
+
+ /** Creates a serializer for the type of sink's input. */
+ TypeSerializer createInputSerializer();
+
+ /**
+ * Returns a metadata consumer, the {@link SinkWriter} can publish metadata events of type
+ * {@link MetaT} to the consumer.
+ *
+ * It is recommended to use a separate thread pool to publish the metadata because enqueuing
+ * a lot of these messages in the mailbox may lead to a performance decrease. thread, and the
+ * {@link Consumer#accept} method is executed very fast.
+ */
+ default Optional> metadataConsumer() {
+ return Optional.empty();
+ }
+}
diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
new file mode 100644
index 000000000000..7ddd6a04392a
--- /dev/null
+++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+import org.apache.flink.configuration.description.TextElement;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+
+import java.time.Duration;
+
+/** A collection of all configuration options that relate to checkpoints and savepoints. */
+public class CheckpointingOptions {
+
+ // ------------------------------------------------------------------------
+ // general checkpoint options
+ // ------------------------------------------------------------------------
+
+ /**
+ * The checkpoint storage used to store operator state locally within the cluster during
+ * execution.
+ *
+ * The implementation can be specified either via their shortcut name, or via the class name
+ * of a {@code StateBackendFactory}. If a StateBackendFactory class name is specified, the
+ * factory is instantiated (via its zero-argument constructor) and its {@code
+ * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called.
+ *
+ *
Recognized shortcut names are 'hashmap' and 'rocksdb'.
+ *
+ * @deprecated Use {@link StateBackendOptions#STATE_BACKEND}.
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS)
+ @Documentation.ExcludeFromDocumentation("Hidden for deprecated")
+ @Deprecated
+ public static final ConfigOption STATE_BACKEND =
+ ConfigOptions.key("state.backend.type")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("state.backend")
+ .withDescription(
+ Description.builder()
+ .text("The state backend to be used to store state.")
+ .linebreak()
+ .text(
+ "The implementation can be specified either via their shortcut "
+ + " name, or via the class name of a %s. "
+ + "If a factory is specified it is instantiated via its "
+ + "zero argument constructor and its %s "
+ + "method is called.",
+ TextElement.code("StateBackendFactory"),
+ TextElement.code(
+ "StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text("Recognized shortcut names are 'hashmap' and 'rocksdb'.")
+ .build());
+
+ /**
+ * The checkpoint storage used to checkpoint state for recovery.
+ *
+ * The implementation can be specified either via their shortcut name, or via the class name
+ * of a {@code CheckpointStorageFactory}. If a CheckpointStorageFactory class name is specified,
+ * the factory is instantiated (via its zero-argument constructor) and its {@code
+ * CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called.
+ *
+ *
Recognized shortcut names are 'jobmanager' and 'filesystem'.
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
+ public static final ConfigOption CHECKPOINT_STORAGE =
+ ConfigOptions.key("state.checkpoint-storage")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The checkpoint storage implementation to be used to checkpoint state.")
+ .linebreak()
+ .text(
+ "The implementation can be specified either via their shortcut "
+ + " name, or via the class name of a %s. "
+ + "If a factory is specified it is instantiated via its "
+ + "zero argument constructor and its %s "
+ + " method is called.",
+ TextElement.code("CheckpointStorageFactory"),
+ TextElement.code(
+ "CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)"))
+ .linebreak()
+ .text(
+ "Recognized shortcut names are 'jobmanager' and 'filesystem'.")
+ .build());
+
+ /** The maximum number of completed checkpoints to retain. */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption MAX_RETAINED_CHECKPOINTS =
+ ConfigOptions.key("state.checkpoints.num-retained")
+ .intType()
+ .defaultValue(1)
+ .withDescription("The maximum number of completed checkpoints to retain.");
+
+ /** @deprecated Checkpoints are always asynchronous. */
+ @Deprecated
+ public static final ConfigOption ASYNC_SNAPSHOTS =
+ ConfigOptions.key("state.backend.async")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription("Deprecated option. All state snapshots are asynchronous.");
+
+ /**
+ * Option whether the state backend should create incremental checkpoints, if possible. For an
+ * incremental checkpoint, only a diff from the previous checkpoint is stored, rather than the
+ * complete checkpoint state.
+ *
+ * Once enabled, the state size shown in web UI or fetched from rest API only represents the
+ * delta checkpoint size instead of full checkpoint size.
+ *
+ *
Some state backends may not support incremental checkpoints and ignore this option.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption INCREMENTAL_CHECKPOINTS =
+ ConfigOptions.key("state.backend.incremental")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "Option whether the state backend should create incremental checkpoints, if possible. For"
+ + " an incremental checkpoint, only a diff from the previous checkpoint is stored, rather than the"
+ + " complete checkpoint state. Once enabled, the state size shown in web UI or fetched from rest API"
+ + " only represents the delta checkpoint size instead of full checkpoint size."
+ + " Some state backends may not support incremental checkpoints and ignore this option.");
+
+ /**
+ * This option configures local recovery for this state backend. By default, local recovery is
+ * deactivated.
+ *
+ * Local recovery currently only covers keyed state backends (including both the
+ * EmbeddedRocksDBStateBackend and the HashMapStateBackend).
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption LOCAL_RECOVERY =
+ ConfigOptions.key("state.backend.local-recovery")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "This option configures local recovery for this state backend. By default, local recovery is "
+ + "deactivated. Local recovery currently only covers keyed state backends "
+ + "(including both the EmbeddedRocksDBStateBackend and the HashMapStateBackend).");
+
+ /**
+ * The config parameter defining the root directories for storing file-based state for local
+ * recovery.
+ *
+ * Local recovery currently only covers keyed state backends. Currently, MemoryStateBackend
+ * does not support local recovery and ignore this option.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS =
+ ConfigOptions.key("taskmanager.state.local.root-dirs")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The config parameter defining the root directories for storing file-based "
+ + "state for local recovery. Local recovery currently only covers keyed "
+ + "state backends. If not configured it will default to /localState. "
+ + "The can be configured via %s",
+ TextElement.code(
+ ClusterOptions
+ .TASK_MANAGER_PROCESS_WORKING_DIR_BASE
+ .key()))
+ .build());
+
+ // ------------------------------------------------------------------------
+ // Options specific to the file-system-based state backends
+ // ------------------------------------------------------------------------
+
+ /**
+ * The default directory for savepoints. Used by the state backends that write savepoints to
+ * file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 3)
+ public static final ConfigOption SAVEPOINT_DIRECTORY =
+ ConfigOptions.key("state.savepoints.dir")
+ .stringType()
+ .noDefaultValue()
+ .withDeprecatedKeys("savepoints.state.backend.fs.dir")
+ .withDescription(
+ "The default directory for savepoints. Used by the state backends that write savepoints to"
+ + " file systems (HashMapStateBackend, EmbeddedRocksDBStateBackend).");
+
+ /**
+ * The default directory used for storing the data files and meta data of checkpoints in a Flink
+ * supported filesystem. The storage path must be accessible from all participating
+ * processes/nodes(i.e. all TaskManagers and JobManagers).
+ */
+ @Documentation.Section(value = Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
+ public static final ConfigOption