From 80524c8ed95e8c9a91336e31a1b86b39bebb9b9d Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Thu, 21 Nov 2024 13:12:23 +0800 Subject: [PATCH 01/17] [flink][hotfix] Wait for consumer reset before job close --- .../apache/paimon/flink/BranchSqlITCase.java | 13 +++++++-- .../paimon/flink/CatalogTableITCase.java | 12 +++++++- .../flink/ContinuousFileStoreITCase.java | 9 +++++- .../flink/action/ConsumerActionITCase.java | 28 ++++++++++++++----- 4 files changed, 51 insertions(+), 11 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java index c25d99cb4459..2566fbe92e4c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.ArrayList; @@ -440,6 +441,7 @@ public void testBranchTagsTable() throws Exception { } @Test + @Timeout(60) public void testBranchConsumersTable() throws Exception { sql("CREATE TABLE t (a INT, b INT)"); sql("INSERT INTO t VALUES (1, 2), (3,4)"); @@ -451,11 +453,18 @@ public void testBranchConsumersTable() throws Exception { "SELECT * FROM t$branch_b1 /*+ OPTIONS('consumer-id'='id1','consumer.expiration-time'='3h') */")); sql("INSERT INTO t$branch_b1 VALUES (5, 6), (7, 8)"); assertThat(iterator.collect(2)).containsExactlyInAnyOrder(Row.of(5, 6), Row.of(7, 8)); + List branchResult; + do { + branchResult = collectResult("SELECT * FROM t$branch_b1$consumers"); + if (!branchResult.isEmpty()) { + break; + } + Thread.sleep(1000); + } while (true); iterator.close(); assertThat(collectResult("SELECT * FROM t$consumers")).isEmpty(); - assertThat(collectResult("SELECT * FROM t$branch_b1$consumers")) - .containsExactlyInAnyOrder("+I[id1, 2]"); + assertThat(branchResult).containsExactlyInAnyOrder("+I[id1, 2]"); assertThat(collectResult("SELECT * FROM t$consumers /*+ OPTIONS('branch'='b1') */")) .containsExactlyInAnyOrder("+I[id1, 2]"); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java index 8a3e068a72a0..2a855796d8d4 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java @@ -33,6 +33,7 @@ import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; import org.apache.flink.types.Row; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import javax.annotation.Nonnull; @@ -940,6 +941,7 @@ public void testTagsTable() throws Exception { } @Test + @Timeout(60) public void testConsumersTable() throws Exception { batchSql("CREATE TABLE T (a INT, b INT)"); batchSql("INSERT INTO T VALUES (1, 2)"); @@ -952,9 +954,17 @@ public void testConsumersTable() throws Exception { batchSql("INSERT INTO T VALUES (5, 6), (7, 8)"); assertThat(iterator.collect(2)).containsExactlyInAnyOrder(Row.of(1, 2), Row.of(3, 4)); + + List result; + do { + result = sql("SELECT * FROM T$consumers"); + if (!result.isEmpty()) { + break; + } + Thread.sleep(1000); + } while (true); iterator.close(); - List result = sql("SELECT * FROM T$consumers"); assertThat(result).hasSize(1); assertThat(result.get(0).getField(0)).isEqualTo("my1"); assertThat((Long) result.get(0).getField(1)).isGreaterThanOrEqualTo(3); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java index 2e15697511dd..b44885832804 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -120,7 +120,14 @@ public void testConsumerId() throws Exception { assertThat(iterator.collect(2)) .containsExactlyInAnyOrder(Row.of("1", "2", "3"), Row.of("4", "5", "6")); - Thread.sleep(1000); + List result; + do { + result = sql("SELECT * FROM %s$consumers", table); + if (!result.isEmpty()) { + break; + } + Thread.sleep(1000); + } while (true); iterator.close(); iterator = diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java index e2243ddf269a..6fb8c81eb744 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java @@ -26,8 +26,11 @@ import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.table.api.TableException; +import org.apache.flink.types.Row; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -46,6 +49,7 @@ public class ConsumerActionITCase extends ActionITCaseBase { @ParameterizedTest + @Timeout(60) @ValueSource(strings = {"action", "procedure_indexed", "procedure_named"}) public void testResetConsumer(String invoker) throws Exception { init(warehouse); @@ -72,18 +76,22 @@ public void testResetConsumer(String invoker) throws Exception { writeData(rowData(3L, BinaryString.fromString("Paimon"))); // use consumer streaming read table - testStreamingRead( + BlockingIterator iterator = + testStreamingRead( "SELECT * FROM `" + tableName + "` /*+ OPTIONS('consumer-id'='myid','consumer.expiration-time'='3h') */", Arrays.asList( changelogRow("+I", 1L, "Hi"), changelogRow("+I", 2L, "Hello"), - changelogRow("+I", 3L, "Paimon"))) - .close(); + changelogRow("+I", 3L, "Paimon"))); - Thread.sleep(1000); ConsumerManager consumerManager = new ConsumerManager(table.fileIO(), table.location()); + while (!consumerManager.consumer("myid").isPresent()) { + Thread.sleep(1000); + } + iterator.close(); + Optional consumer1 = consumerManager.consumer("myid"); assertThat(consumer1).isPresent(); assertThat(consumer1.get().nextSnapshot()).isEqualTo(4); @@ -191,6 +199,7 @@ public void testResetConsumer(String invoker) throws Exception { } @ParameterizedTest + @Timeout(60) @ValueSource(strings = {"action", "procedure_indexed", "procedure_named"}) public void testResetBranchConsumer(String invoker) throws Exception { init(warehouse); @@ -222,18 +231,23 @@ public void testResetBranchConsumer(String invoker) throws Exception { String branchTableName = tableName + "$branch_b1"; // use consumer streaming read table - testStreamingRead( + BlockingIterator iterator = + testStreamingRead( "SELECT * FROM `" + branchTableName + "` /*+ OPTIONS('consumer-id'='myid','consumer.expiration-time'='3h') */", Arrays.asList( changelogRow("+I", 1L, "Hi"), changelogRow("+I", 2L, "Hello"), - changelogRow("+I", 3L, "Paimon"))) - .close(); + changelogRow("+I", 3L, "Paimon"))); ConsumerManager consumerManager = new ConsumerManager(table.fileIO(), table.location(), branchName); + while (!consumerManager.consumer("myid").isPresent()) { + Thread.sleep(1000); + } + iterator.close(); + Optional consumer1 = consumerManager.consumer("myid"); assertThat(consumer1).isPresent(); assertThat(consumer1.get().nextSnapshot()).isEqualTo(4); From a4d79e3ca7f7f185eec498bb73b442d5f482ba3a Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Thu, 21 Nov 2024 14:26:06 +0800 Subject: [PATCH 02/17] [flink][cdc] Update flink dependency to 1.20 --- paimon-flink/paimon-flink-cdc/pom.xml | 11 ++++-- .../streamrecord/RecordAttributes.java | 22 ------------ .../catalog/CatalogMaterializedTable.java | 34 ------------------- 3 files changed, 9 insertions(+), 58 deletions(-) delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java diff --git a/paimon-flink/paimon-flink-cdc/pom.xml b/paimon-flink/paimon-flink-cdc/pom.xml index 7f5bb3e786da..792c6c14378b 100644 --- a/paimon-flink/paimon-flink-cdc/pom.xml +++ b/paimon-flink/paimon-flink-cdc/pom.xml @@ -34,7 +34,7 @@ under the License. Paimon : Flink : CDC - 1.18.1 + 1.20.0 3.1.1 3.1.1 1.11.4 @@ -43,7 +43,7 @@ under the License. 1.19.1 4.0.0-1.17 7.5.0 - 3.0.1-1.18 + 3.3.0-1.20 @@ -167,6 +167,13 @@ under the License. + + commons-codec + commons-codec + 1.9 + test + + org.apache.paimon paimon-common diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java deleted file mode 100644 index 723c71dc565d..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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.runtime.streamrecord; - -/** Placeholder class for new feature introduced since flink 1.19. Should never be used. */ -public class RecordAttributes extends StreamElement {} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java deleted file mode 100644 index 6eabd1db7f38..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.catalog; - -/** - * Dummy placeholder to resolve compatibility issue of CatalogMaterializedTable(introduced in flink - * 1.20). - */ -public interface CatalogMaterializedTable extends CatalogBaseTable { - /** Dummy LogicalRefreshMode placeholder. */ - enum LogicalRefreshMode {} - - /** Dummy RefreshMode placeholder. */ - enum RefreshMode {} - - /** Dummy RefreshStatus placeholder. */ - enum RefreshStatus {} -} From 2c6ea46f0d355d5a72c1e212ddcaf3f80006ccb6 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Thu, 21 Nov 2024 22:50:39 +0800 Subject: [PATCH 03/17] [flink] Adopt open(OpenContext) in RichFunction --- .../common/functions/DefaultOpenContext.java | 28 +++ .../api/common/functions/OpenContext.java | 29 +++ .../api/common/functions/RichFunction.java | 171 ++++++++++++++++++ .../common/functions/DefaultOpenContext.java | 28 +++ .../api/common/functions/OpenContext.java | 29 +++ .../api/common/functions/RichFunction.java | 171 ++++++++++++++++++ .../common/functions/DefaultOpenContext.java | 28 +++ .../api/common/functions/OpenContext.java | 29 +++ .../api/common/functions/RichFunction.java | 171 ++++++++++++++++++ .../common/functions/DefaultOpenContext.java | 28 +++ .../api/common/functions/OpenContext.java | 29 +++ .../api/common/functions/RichFunction.java | 171 ++++++++++++++++++ .../paimon/flink/kafka/KafkaSinkFunction.java | 12 +- ...CdcDynamicTableParsingProcessFunction.java | 12 +- .../CdcMultiTableParsingProcessFunction.java | 12 +- .../sink/cdc/CdcParsingProcessFunction.java | 12 +- .../UpdatedDataFieldsProcessFunctionBase.java | 12 +- .../flink/service/QueryAddressRegister.java | 12 +- .../flink/service/QueryFileMonitor.java | 12 +- .../paimon/flink/shuffle/RangeShuffle.java | 15 +- .../flink/sink/RowDataStoreWriteOperator.java | 4 +- .../apache/paimon/flink/sorter/SortUtils.java | 28 ++- .../source/BucketUnawareCompactSource.java | 12 +- .../CombinedAwareBatchSourceFunction.java | 12 +- .../CombinedAwareStreamingSourceFunction.java | 12 +- .../CombinedCompactorSourceFunction.java | 12 +- .../CombinedUnawareBatchSourceFunction.java | 12 +- ...ombinedUnawareStreamingSourceFunction.java | 12 +- 28 files changed, 1095 insertions(+), 20 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/RichFunction.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/RichFunction.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/RichFunction.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/DefaultOpenContext.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/RichFunction.java 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-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.15/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.15/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.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.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.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.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-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java index 72a177adceaf..41e7141cf48a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java @@ -21,6 +21,7 @@ import org.apache.paimon.flink.sink.LogSinkFunction; import org.apache.paimon.table.sink.SinkRecord; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaException; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; @@ -65,7 +66,16 @@ public void setWriteCallback(WriteCallback writeCallback) { this.writeCallback = writeCallback; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration configuration) throws Exception { super.open(configuration); Callback baseCallback = requireNonNull(callback); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java index 0961ff160048..886e33e2046a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -74,7 +75,16 @@ public CdcDynamicTableParsingProcessFunction( this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); catalog = catalogLoader.load(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java index b18a05c280cb..4c5e0600bb47 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java @@ -20,6 +20,7 @@ import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.configuration.Configuration; @@ -51,7 +52,16 @@ public CdcMultiTableParsingProcessFunction(EventParser.Factory parserFactory) this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); updatedDataFieldsOutputTags = new HashMap<>(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java index 3456634942c8..eec228f3c09b 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java @@ -20,6 +20,7 @@ import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -50,7 +51,16 @@ public CdcParsingProcessFunction(EventParser.Factory parserFactory) { this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java index c2e928bd4a0a..4f02b784c2ba 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java @@ -31,6 +31,7 @@ import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.StringUtils; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.slf4j.Logger; @@ -73,7 +74,16 @@ protected UpdatedDataFieldsProcessFunctionBase(Catalog.Loader catalogLoader) { this.catalogLoader = catalogLoader; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) { this.catalog = catalogLoader.load(); this.allowUpperCase = this.catalog.allowUpperCase(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java index df3cf7abf2a5..524f2e5f01c1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java @@ -23,6 +23,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.sink.SinkFunction; @@ -44,7 +45,16 @@ public QueryAddressRegister(Table table) { this.serviceManager = ((FileStoreTable) table).store().newServiceManager(); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { this.executors = new TreeMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java index 43cf654e91fe..02f8a654112e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java @@ -31,6 +31,7 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.table.system.FileMonitorTable; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -70,7 +71,16 @@ public QueryFileMonitor(Table table) { .toMillis(); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); ReadBuilder readBuilder = monitorTable.newReadBuilder(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java index 54104130438b..8760f1dc5f80 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java @@ -27,6 +27,7 @@ import org.apache.paimon.utils.SerializableSupplier; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; @@ -182,9 +183,19 @@ public KeyAndSizeExtractor(RowType rowType, boolean isSortBySize) { this.isSortBySize = isSortBySize; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink + * 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink + * 2.0+. + */ public void open(Configuration parameters) throws Exception { - super.open(parameters); InternalRowToSizeVisitor internalRowToSizeVisitor = new InternalRowToSizeVisitor(); fieldSizeCalculator = rowType.getFieldTypes().stream() diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java index 07fe275543a1..e3ada5363587 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java @@ -23,10 +23,10 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.SinkRecord; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.functions.sink.SinkFunction; @@ -100,7 +100,7 @@ public void open() throws Exception { // to stay compatible with Flink 1.18- if (logSinkFunction instanceof RichFunction) { RichFunction richFunction = (RichFunction) logSinkFunction; - richFunction.open(new Configuration()); + richFunction.open(DefaultOpenContext.INSTANCE); } logCallback = new LogWriteCallback(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java index f590c2fb7fff..b30e14551296 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java @@ -31,6 +31,7 @@ import org.apache.paimon.utils.KeyProjectedRow; import org.apache.paimon.utils.SerializableSupplier; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -119,9 +120,19 @@ public static DataStream sortStreamByKey( .map( new RichMapFunction>() { - @Override + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { - super.open(parameters); shuffleKeyAbstract.open(); } @@ -172,7 +183,18 @@ public Tuple2 map(RowData value) { private transient KeyProjectedRow keyProjectedRow; - @Override + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 2.0+. + */ public void open(Configuration parameters) { keyProjectedRow = new KeyProjectedRow(valueProjectionMap); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java index d306c7d8e1e5..e768c717ddaa 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java @@ -26,6 +26,7 @@ import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.utils.Preconditions; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStreamSource; @@ -72,7 +73,16 @@ public BucketUnawareCompactSource( this.filter = filter; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { compactionCoordinator = new UnawareAppendTableCompactionCoordinator(table, streaming, filter); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java index cee6081aa29f..2157be51aee4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java @@ -25,6 +25,7 @@ import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -62,7 +63,16 @@ public CombinedAwareBatchSourceFunction( super(catalogLoader, includingPattern, excludingPattern, databasePattern, false); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { super.open(parameters); tableScan = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java index bff690ea30c2..01e0127e9fda 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java @@ -25,6 +25,7 @@ import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -59,7 +60,16 @@ public CombinedAwareStreamingSourceFunction( this.monitorInterval = monitorInterval; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { super.open(parameters); tableScan = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java index 1964927b5cdd..02bb8786505d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.table.source.Split; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; @@ -70,7 +71,16 @@ public CombinedCompactorSourceFunction( this.isStreaming = isStreaming; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { isRunning = new AtomicBoolean(true); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java index 8ec8d5f2c1a2..6a40f10ada61 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java @@ -29,6 +29,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; @@ -69,7 +70,16 @@ public CombinedUnawareBatchSourceFunction( super(catalogLoader, includingPattern, excludingPattern, databasePattern, false); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { super.open(parameters); tableScan = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java index e398e09a8451..b64518a7ef60 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java @@ -24,6 +24,7 @@ import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; @@ -55,7 +56,16 @@ public CombinedUnawareStreamingSourceFunction( this.monitorInterval = monitorInterval; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { super.open(parameters); tableScan = From 0a0f42cfc80bcb6725e112154ad02eff89e1f207 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:33:08 +0800 Subject: [PATCH 04/17] [flink] Adopt getTaskInfo() when acquiring parallelism info --- .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../flink/service/QueryExecutorOperator.java | 10 +++--- .../paimon/flink/sink/CommitterOperator.java | 5 ++- .../sink/HashBucketAssignerOperator.java | 5 +-- .../sink/MultiTablesStoreCompactOperator.java | 7 ++-- .../flink/sink/StoreCompactOperator.java | 7 ++-- .../paimon/flink/sink/TableWriteOperator.java | 5 +-- .../index/GlobalIndexAssignerOperator.java | 5 +-- .../sink/index/IndexBootstrapOperator.java | 5 +-- .../paimon/flink/sorter/SortOperator.java | 4 ++- .../AppendBypassCoordinateOperator.java | 3 +- .../source/BucketUnawareCompactSource.java | 3 +- .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../UnawareBucketAppendOnlyTableITCase.java | 3 +- 17 files changed, 201 insertions(+), 21 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java 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.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.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.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.18/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.18/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-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java index 556c30839688..bf0521d55049 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.service.network.NetworkUtils; @@ -77,8 +78,8 @@ public void initializeState(StateInitializationContext context) throws Exception this.query = ((FileStoreTable) table).newLocalTableQuery().withIOManager(ioManager); KvQueryServer server = new KvQueryServer( - getRuntimeContext().getIndexOfThisSubtask(), - getRuntimeContext().getNumberOfParallelSubtasks(), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), NetworkUtils.findHostAddress(), Collections.singletonList(0).iterator(), 1, @@ -96,8 +97,9 @@ public void initializeState(StateInitializationContext context) throws Exception this.output.collect( new StreamRecord<>( GenericRow.of( - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), BinaryString.fromString(address.getHostName()), address.getPort()))); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java index 2ec90b8c6c40..021a5db413d5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.utils.Preconditions; import org.apache.flink.runtime.state.StateInitializationContext; @@ -129,7 +130,9 @@ public void initializeState(StateInitializationContext context) throws Exception super.initializeState(context); Preconditions.checkArgument( - !forceSingleParallelism || getRuntimeContext().getNumberOfParallelSubtasks() == 1, + !forceSingleParallelism + || RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) + == 1, "Committer Operator parallelism in paimon MUST be one."); this.currentWatermark = Long.MIN_VALUE; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java index 70fac7a83e93..0c101c6d1e01 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.flink.ProcessRecordAttributesUtil; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.index.BucketAssigner; import org.apache.paimon.index.HashBucketAssigner; import org.apache.paimon.index.SimpleHashBucketAssigner; @@ -76,8 +77,8 @@ public void initializeState(StateInitializationContext context) throws Exception StateUtils.getSingleValueFromState( context, "commit_user_state", String.class, initialCommitUser); - int numberTasks = getRuntimeContext().getNumberOfParallelSubtasks(); - int taskId = getRuntimeContext().getIndexOfThisSubtask(); + int numberTasks = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); + int taskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); long targetRowNum = table.coreOptions().dynamicBucketTargetRowNum(); this.assigner = overwrite diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java index 7cb5d30c2f8e..8a1d3a02df81 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.options.Options; @@ -109,8 +110,10 @@ public void initializeState(StateInitializationContext context) throws Exception ChannelComputer.select( partition, bucket, - getRuntimeContext().getNumberOfParallelSubtasks()) - == getRuntimeContext().getIndexOfThisSubtask()); + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext())) + == RuntimeContextUtils.getIndexOfThisSubtask( + getRuntimeContext())); tables = new HashMap<>(); writes = new HashMap<>(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java index bc7bb350df21..9b9c3fd2ed68 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.options.Options; @@ -89,8 +90,10 @@ public void initializeState(StateInitializationContext context) throws Exception ChannelComputer.select( partition, bucket, - getRuntimeContext().getNumberOfParallelSubtasks()) - == getRuntimeContext().getIndexOfThisSubtask()); + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext())) + == RuntimeContextUtils.getIndexOfThisSubtask( + getRuntimeContext())); write = storeSinkWriteProvider.provide( table, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java index 67b4720e2964..32fcdd03bdfd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java @@ -21,6 +21,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.flink.ProcessRecordAttributesUtil; import org.apache.paimon.flink.sink.StoreSinkWriteState.StateValueFilter; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.ChannelComputer; @@ -58,14 +59,14 @@ public void initializeState(StateInitializationContext context) throws Exception super.initializeState(context); boolean containLogSystem = containLogSystem(); - int numTasks = getRuntimeContext().getNumberOfParallelSubtasks(); + int numTasks = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); StateValueFilter stateFilter = (tableName, partition, bucket) -> { int task = containLogSystem ? ChannelComputer.select(bucket, numTasks) : ChannelComputer.select(partition, bucket, numTasks); - return task == getRuntimeContext().getIndexOfThisSubtask(); + return task == RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); }; state = createState(context, stateFilter); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java index 7fee3f45f3db..99cce07fdc57 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java @@ -22,6 +22,7 @@ import org.apache.paimon.crosspartition.KeyPartOrRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.table.Table; import org.apache.flink.api.java.tuple.Tuple2; @@ -59,8 +60,8 @@ public void initializeState(StateInitializationContext context) throws Exception assigner.open( computeManagedMemory(this), ioManager, - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), this::collect); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java index 501e35dff46c..5c8ba8f9441f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java @@ -21,6 +21,7 @@ import org.apache.paimon.crosspartition.IndexBootstrap; import org.apache.paimon.crosspartition.KeyPartOrRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.utils.SerializableFunction; import org.apache.flink.api.java.tuple.Tuple2; @@ -50,8 +51,8 @@ public IndexBootstrapOperator( public void initializeState(StateInitializationContext context) throws Exception { super.initializeState(context); bootstrap.bootstrap( - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), this::collect); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java index d4d5dd741681..b6847125fbc6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.options.MemorySize; import org.apache.paimon.sort.BinaryExternalSortBuffer; import org.apache.paimon.types.RowType; @@ -79,7 +80,8 @@ public SortOperator( public void open() throws Exception { super.open(); initBuffer(); - if (sinkParallelism != getRuntimeContext().getNumberOfParallelSubtasks()) { + if (sinkParallelism + != RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext())) { throw new IllegalArgumentException( "Please ensure that the runtime parallelism of the sink matches the initial configuration " + "to avoid potential issues with skewed range partitioning."); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java index 668aa24c145d..45090f7b68b4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.ExecutorUtils; @@ -67,7 +68,7 @@ public AppendBypassCoordinateOperator( public void open() throws Exception { super.open(); checkArgument( - getRuntimeContext().getNumberOfParallelSubtasks() == 1, + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) == 1, "Compaction Coordinator parallelism in paimon MUST be one."); long intervalMs = table.coreOptions().continuousDiscoveryInterval().toMillis(); this.compactTasks = new LinkedBlockingQueue<>(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java index e768c717ddaa..79ee827fe6e4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java @@ -21,6 +21,7 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; import org.apache.paimon.flink.sink.CompactionTaskTypeInfo; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.EndOfScanException; @@ -87,7 +88,7 @@ public void open(Configuration parameters) throws Exception { compactionCoordinator = new UnawareAppendTableCompactionCoordinator(table, streaming, filter); Preconditions.checkArgument( - this.getRuntimeContext().getNumberOfParallelSubtasks() == 1, + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) == 1, "Compaction Operator parallelism in paimon MUST be one."); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 000000000000..34e0d041b6a0 --- /dev/null +++ b/paimon-flink/paimon-flink-common/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.getTaskInfo().getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getTaskInfo().getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java index cb323542d4c1..f6dfb1b23046 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.reader.RecordReader; @@ -395,7 +396,7 @@ private TestStatelessWriterSource(FileStoreTable table) { @Override public void run(SourceContext sourceContext) throws Exception { - int taskId = getRuntimeContext().getIndexOfThisSubtask(); + int taskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); // wait some time in parallelism #2, // so that it does not commit in the same checkpoint with parallelism #1 int waitCount = (taskId == 0 ? 0 : 10); From 1778354ba558386aed10db3c2e69422b54715d1e Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Thu, 21 Nov 2024 16:30:12 +0800 Subject: [PATCH 05/17] [flink] Avoid deprecated usages about Configuration --- .../apache/paimon/benchmark/QueryRunner.java | 2 +- .../serialization/SerializerConfig.java | 224 +++++++++++++ .../configuration/CheckpointingOptions.java | 264 +++++++++++++++ .../configuration/StateRecoveryOptions.java | 39 +++ .../sink/cdc/CdcRecordSerializeITCase.java | 148 +++++++++ .../serialization/SerializerConfig.java | 224 +++++++++++++ .../configuration/CheckpointingOptions.java | 264 +++++++++++++++ .../configuration/StateRecoveryOptions.java | 39 +++ .../sink/cdc/CdcRecordSerializeITCase.java | 148 +++++++++ .../serialization/SerializerConfig.java | 224 +++++++++++++ .../configuration/CheckpointingOptions.java | 265 +++++++++++++++ .../configuration/StateRecoveryOptions.java | 39 +++ .../sink/cdc/CdcRecordSerializeITCase.java | 148 +++++++++ .../serialization/SerializerConfig.java | 224 +++++++++++++ .../configuration/CheckpointingOptions.java | 265 +++++++++++++++ .../configuration/StateRecoveryOptions.java | 39 +++ .../sink/cdc/CdcRecordSerializeITCase.java | 148 +++++++++ .../configuration/CheckpointingOptions.java | 304 ++++++++++++++++++ .../configuration/StateRecoveryOptions.java | 39 +++ ...afkaDebeziumAvroDeserializationSchema.java | 2 +- .../strategy/MongoVersionStrategy.java | 8 +- .../action/cdc/mysql/MySqlRecordParser.java | 15 +- ...lsarDebeziumAvroDeserializationSchema.java | 2 +- .../cdc/mongodb/MongodbSchemaITCase.java | 67 ++-- .../cdc/mysql/MySqlSyncTableActionITCase.java | 10 +- .../sink/cdc/CdcRecordSerializeITCase.java | 4 +- .../changelog/ChangelogTaskTypeInfo.java | 13 +- .../flink/sink/CommittableTypeInfo.java | 12 +- .../flink/sink/CompactionTaskTypeInfo.java | 12 +- .../apache/paimon/flink/sink/FlinkSink.java | 6 +- .../sink/MultiTableCommittableTypeInfo.java | 12 +- .../MultiTableCompactionTaskTypeInfo.java | 13 +- .../flink/source/FlinkSourceBuilder.java | 12 +- .../AlignedContinuousFileStoreSource.java | 2 +- .../paimon/flink/utils/InternalTypeInfo.java | 14 +- .../paimon/flink/utils/JavaTypeInfo.java | 16 +- .../paimon/flink/FileSystemCatalogITCase.java | 4 +- .../paimon/flink/RescaleBucketITCase.java | 6 +- .../UnawareBucketAppendOnlyTableITCase.java | 2 +- .../flink/sink/SinkSavepointITCase.java | 2 +- .../paimon/flink/util/AbstractTestBase.java | 4 +- .../flink/util/ReadWriteTableTestUtil.java | 26 +- .../paimon/hive/HiveCatalogITCaseBase.java | 4 +- 43 files changed, 3222 insertions(+), 93 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java create mode 100644 paimon-flink/paimon-flink-1.15/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java create mode 100644 paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java create mode 100644 paimon-flink/paimon-flink-1.17/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java create mode 100644 paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java create mode 100644 paimon-flink/paimon-flink-1.19/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java create mode 100644 paimon-flink/paimon-flink-1.19/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java 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/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> 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 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>> + 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>> getDefaultKryoSerializerClasses(); + + /** Returns the registered Kryo types. */ + LinkedHashSet> getRegisteredKryoTypes(); + + /** Returns the registered POJO types. */ + LinkedHashSet> getRegisteredPojoTypes(); + + /** Returns the registered type info factories. */ + Map, Class>> 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/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/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/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> 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 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>> + 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>> getDefaultKryoSerializerClasses(); + + /** Returns the registered Kryo types. */ + LinkedHashSet> getRegisteredKryoTypes(); + + /** Returns the registered POJO types. */ + LinkedHashSet> getRegisteredPojoTypes(); + + /** Returns the registered type info factories. */ + Map, Class>> 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/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/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/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> 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 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>> + 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>> getDefaultKryoSerializerClasses(); + + /** Returns the registered Kryo types. */ + LinkedHashSet> getRegisteredKryoTypes(); + + /** Returns the registered POJO types. */ + LinkedHashSet> getRegisteredPojoTypes(); + + /** Returns the registered type info factories. */ + Map, Class>> 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/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/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/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> 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 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>> + 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>> getDefaultKryoSerializerClasses(); + + /** Returns the registered Kryo types. */ + LinkedHashSet> getRegisteredKryoTypes(); + + /** Returns the registered POJO types. */ + LinkedHashSet> getRegisteredPojoTypes(); + + /** Returns the registered type info factories. */ + Map, Class>> 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/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 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.18/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java new file mode 100644 index 000000000000..2193c2a6aab9 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/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.18/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java b/paimon-flink/paimon-flink-1.18/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.18/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.19/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java new file mode 100644 index 000000000000..6f1e399d2119 --- /dev/null +++ b/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java @@ -0,0 +1,304 @@ +/* + * 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."); + + /* Option whether to clean individual checkpoint's operatorstates in parallel. If enabled, + * operator states are discarded in parallel using the ExecutorService passed to the cleaner. + * This speeds up checkpoints cleaning, but adds load to the IO. + */ + @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS) + public static final ConfigOption CLEANER_PARALLEL_MODE = + ConfigOptions.key("state.checkpoint.cleaner.parallel-mode") + .booleanType() + .defaultValue(true) + .withDescription( + "Option whether to discard a checkpoint's states in parallel using" + + " the ExecutorService passed into the cleaner"); + + /** @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)."); + + /** + * Whether to create sub-directories named by job id to store the data files and meta data of + * checkpoints. The default value is true to enable user could run several jobs with the same + * checkpoint directory at the same time. If this value is set to false, pay attention not to + * run several jobs with the same directory simultaneously. + */ + @Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS) + public static final ConfigOption CREATE_CHECKPOINT_SUB_DIR = + ConfigOptions.key("state.checkpoints.create-subdir") + .booleanType() + .defaultValue(true) + .withDescription( + Description.builder() + .text( + "Whether to create sub-directories named by job id under the '%s' to store the data files and meta data " + + "of checkpoints. The default value is true to enable user could run several jobs with the same " + + "checkpoint directory at the same time. If this value is set to false, pay attention not to " + + "run several jobs with the same directory simultaneously. ", + TextElement.code(CHECKPOINTS_DIRECTORY.key())) + .linebreak() + .text( + "WARNING: This is an advanced configuration. If set to false, users must ensure that no multiple jobs are run " + + "with the same checkpoint directory, and that no files exist other than those necessary for the " + + "restoration of the current job when starting a new job.") + .build()); + + /** + * 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.19/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java b/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/flink/configuration/StateRecoveryOptions.java new file mode 100644 index 000000000000..2193c2a6aab9 --- /dev/null +++ b/paimon-flink/paimon-flink-1.19/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-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java index fc672b9dc0ab..eea364d460de 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java @@ -48,7 +48,7 @@ public class KafkaDebeziumAvroDeserializationSchema public KafkaDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { this.topic = KafkaActionUtils.findOneTopic(cdcSourceConfig); - this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + this.schemaRegistryUrl = cdcSourceConfig.get(SCHEMA_REGISTRY_URL); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java index 64f127571134..df288a4150e6 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java @@ -83,7 +83,7 @@ default Map getExtractRow( Configuration mongodbConfig) throws JsonProcessingException { SchemaAcquisitionMode mode = - SchemaAcquisitionMode.valueOf(mongodbConfig.getString(START_MODE).toUpperCase()); + SchemaAcquisitionMode.valueOf(mongodbConfig.get(START_MODE).toUpperCase()); ObjectNode objectNode = JsonSerdeUtil.asSpecificNodeType(jsonNode.asText(), ObjectNode.class); JsonNode idNode = objectNode.get(ID_FIELD); @@ -92,7 +92,7 @@ default Map getExtractRow( "The provided MongoDB JSON document does not contain an _id field."); } JsonNode document = - mongodbConfig.getBoolean(DEFAULT_ID_GENERATION) + mongodbConfig.get(DEFAULT_ID_GENERATION) ? objectNode.set( ID_FIELD, idNode.get(OID_FIELD) == null ? idNode : idNode.get(OID_FIELD)) @@ -101,8 +101,8 @@ default Map getExtractRow( case SPECIFIED: return parseFieldsFromJsonRecord( document.toString(), - mongodbConfig.getString(PARSER_PATH), - mongodbConfig.getString(FIELD_NAME), + mongodbConfig.get(PARSER_PATH), + mongodbConfig.get(FIELD_NAME), computedColumns, rowTypeBuilder); case DYNAMIC: diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java index 502e6237a477..26579e718f56 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java @@ -45,6 +45,8 @@ import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -99,11 +101,14 @@ public MySqlRecordParser( .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); String stringifyServerTimeZone = mySqlConfig.get(MySqlSourceOptions.SERVER_TIME_ZONE); - this.isDebeziumSchemaCommentsEnabled = - mySqlConfig.getBoolean( - DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX - + RelationalDatabaseConnectorConfig.INCLUDE_SCHEMA_COMMENTS.name(), - false); + ConfigOption includeSchemaCommentsConfig = + ConfigOptions.key( + DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX + + RelationalDatabaseConnectorConfig.INCLUDE_SCHEMA_COMMENTS + .name()) + .booleanType() + .defaultValue(false); + this.isDebeziumSchemaCommentsEnabled = mySqlConfig.get(includeSchemaCommentsConfig); this.serverTimeZone = stringifyServerTimeZone == null ? ZoneId.systemDefault() diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java index b0d1d1bf620f..f45ee034bec8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java @@ -46,7 +46,7 @@ public class PulsarDebeziumAvroDeserializationSchema public PulsarDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { this.topic = PulsarActionUtils.findOneTopic(cdcSourceConfig); - this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + this.schemaRegistryUrl = cdcSourceConfig.get(SCHEMA_REGISTRY_URL); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java index 394cdd1f149b..f0328b566324 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java @@ -76,13 +76,12 @@ public static void initMongoDB() { @Test public void testCreateSchemaFromValidConfig() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); Schema schema = MongodbSchemaUtils.getMongodbSchema(mongodbConfig); assertNotNull(schema); } @@ -90,13 +89,12 @@ public void testCreateSchemaFromValidConfig() { @Test public void testCreateSchemaFromInvalidHost() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, "127.0.0.1:12345"); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, "127.0.0.1:12345"); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); @@ -106,7 +104,7 @@ public void testCreateSchemaFromInvalidHost() { public void testCreateSchemaFromIncompleteConfig() { // Create a Configuration object with missing necessary settings Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); // Expect an exception to be thrown due to missing necessary settings assertThrows( NullPointerException.class, @@ -117,13 +115,12 @@ public void testCreateSchemaFromIncompleteConfig() { public void testCreateSchemaFromDynamicConfig() { // Create a Configuration object with the necessary settings Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); // Call the method and check the results Schema schema = MongodbSchemaUtils.getMongodbSchema(mongodbConfig); @@ -142,13 +139,12 @@ public void testCreateSchemaFromDynamicConfig() { @Test public void testCreateSchemaFromInvalidDatabase() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "invalidDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "invalidDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); @@ -157,13 +153,12 @@ public void testCreateSchemaFromInvalidDatabase() { @Test public void testCreateSchemaFromInvalidCollection() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "invalidCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "invalidCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index bdeab07a746c..febbe4e1deaa 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -31,7 +31,8 @@ import org.apache.paimon.utils.CommonTestUtils; import org.apache.paimon.utils.JsonSerdeUtil; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.BeforeAll; @@ -1285,8 +1286,11 @@ public void testDefaultCheckpointInterval() throws Exception { mySqlConfig.put("database-name", "default_checkpoint"); mySqlConfig.put("table-name", "t"); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setRestartStrategy(RestartStrategies.noRestart()); + // Using `none` to avoid compatibility issues with Flink 1.18-. + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, "none"); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build(); action.withStreamExecutionEnvironment(env); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java index 698900436e8d..d01c331539d6 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java @@ -24,7 +24,7 @@ import org.apache.paimon.types.RowType; import org.apache.paimon.types.VarCharType; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfigImpl; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -102,7 +102,7 @@ public void testUnmodifiableListKryoSerialize() throws IOException { } public static KryoSerializer createFlinkKryoSerializer(Class type) { - return new KryoSerializer<>(type, new ExecutionConfig()); + return new KryoSerializer<>(type, new SerializerConfigImpl()); } private static final class TestOutputView extends DataOutputStream implements DataOutputView { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java index 5cae899a0704..a529e6764fae 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java @@ -21,6 +21,7 @@ import org.apache.paimon.flink.sink.NoneCopyVersionedSerializerTypeSerializerProxy; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -56,7 +57,17 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer( + SerializerConfig serializerConfig) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // we don't need copy for task return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java index dcb87238b833..92e826a91379 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java @@ -21,6 +21,7 @@ import org.apache.paimon.table.sink.CommitMessageSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -57,7 +58,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // no copy, so that data from writer is directly going into committer while chaining return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java index 47defa61a971..6510a85b800a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java @@ -22,6 +22,7 @@ import org.apache.paimon.table.sink.CompactionTaskSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -58,7 +59,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // we don't need copy for task return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index 59f2f4b1035f..28858b85a4a7 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -35,6 +35,7 @@ import org.apache.flink.api.common.operators.SlotSharingGroup; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.typeutils.EitherTypeInfo; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.CheckpointingMode; @@ -42,7 +43,6 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; @@ -338,12 +338,12 @@ public static void assertStreamingConfiguration(StreamExecutionEnvironment env) checkArgument( !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(), "Paimon sink currently does not support unaligned checkpoints. Please set " - + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key() + + CheckpointingOptions.ENABLE_UNALIGNED.key() + " to false."); checkArgument( env.getCheckpointConfig().getCheckpointingMode() == CheckpointingMode.EXACTLY_ONCE, "Paimon sink currently only supports EXACTLY_ONCE checkpoint mode. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key() + + CheckpointingOptions.CHECKPOINTING_CONSISTENCY_MODE.key() + " to exactly-once"); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java index f82f08209867..7da0ae0e2078 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java @@ -21,6 +21,7 @@ import org.apache.paimon.table.sink.CommitMessageSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -57,7 +58,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // no copy, so that data from writer is directly going into committer while chaining return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java index f27f29f87fe7..0116ff198811 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java @@ -23,6 +23,7 @@ import org.apache.paimon.table.sink.MultiTableCompactionTaskSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy; @@ -60,7 +61,17 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer( + SerializerConfig serializerConfig) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer( ExecutionConfig executionConfig) { return new SimpleVersionedSerializerTypeSerializerProxy< diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java index b3dcd4840cc1..6933be18def6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java @@ -41,12 +41,12 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.connector.base.source.hybrid.HybridSource; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; @@ -331,30 +331,30 @@ private void assertStreamingConfigurationForAlignMode(StreamExecutionEnvironment checkArgument( checkpointConfig.isCheckpointingEnabled(), "The align mode of paimon source is only supported when checkpoint enabled. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL.key() + + CheckpointingOptions.CHECKPOINTING_INTERVAL.key() + "larger than 0"); checkArgument( checkpointConfig.getMaxConcurrentCheckpoints() == 1, "The align mode of paimon source supports at most one ongoing checkpoint at the same time. Please set " - + ExecutionCheckpointingOptions.MAX_CONCURRENT_CHECKPOINTS.key() + + CheckpointingOptions.MAX_CONCURRENT_CHECKPOINTS.key() + " to 1"); checkArgument( checkpointConfig.getCheckpointTimeout() > conf.get(FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_TIMEOUT) .toMillis(), "The align mode of paimon source requires that the timeout of checkpoint is greater than the timeout of the source's snapshot alignment. Please increase " - + ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT.key() + + CheckpointingOptions.CHECKPOINTING_TIMEOUT.key() + " or decrease " + FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_TIMEOUT.key()); checkArgument( !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(), "The align mode of paimon source currently does not support unaligned checkpoints. Please set " - + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key() + + CheckpointingOptions.ENABLE_UNALIGNED.key() + " to false."); checkArgument( env.getCheckpointConfig().getCheckpointingMode() == CheckpointingMode.EXACTLY_ONCE, "The align mode of paimon source currently only supports EXACTLY_ONCE checkpoint mode. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key() + + CheckpointingOptions.CHECKPOINTING_CONSISTENCY_MODE.key() + " to exactly-once"); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java index d6b7060763ac..705e1d9a7a4c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java @@ -73,7 +73,7 @@ public SourceReader createReader(SourceReaderCont limit, new FutureCompletingBlockingQueue<>( context.getConfiguration() - .getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY))); + .get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY))); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java index 4ea5db9f34d4..60898421ddea 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java @@ -22,6 +22,7 @@ import org.apache.paimon.types.RowType; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -73,8 +74,17 @@ public boolean isKeyType() { return false; } - @Override - public TypeSerializer createSerializer(ExecutionConfig config) { + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { return serializer.duplicate(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java index a36243c5bdac..4aea809b51bc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -78,7 +79,16 @@ public boolean isKeyType() { return Comparable.class.isAssignableFrom(typeClass); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { return new JavaSerializer<>(this.typeClass); } @@ -91,7 +101,9 @@ public TypeComparator createComparator( @SuppressWarnings("rawtypes") GenericTypeComparator comparator = new GenericTypeComparator( - sortOrderAscending, createSerializer(executionConfig), this.typeClass); + sortOrderAscending, + new JavaSerializer<>(this.typeClass), + this.typeClass); return (TypeComparator) comparator; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java index 239043ff79e1..e10623b95070 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java @@ -27,7 +27,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.utils.BlockingIterator; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; @@ -60,7 +60,7 @@ public void setup() { tableEnvironmentBuilder() .streamingMode() .parallelism(1) - .setConf(ExecutionCheckpointingOptions.ENABLE_UNALIGNED, false) + .setConf(CheckpointingOptions.ENABLE_UNALIGNED, false) .build(); path = getTempDirPath(); tEnv.executeSql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java index 08969bddfdb3..03b015d08f4b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java @@ -24,9 +24,9 @@ import org.apache.paimon.schema.TableSchema; import org.apache.paimon.utils.SnapshotManager; +import org.apache.flink.configuration.StateRecoveryOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.types.Row; import org.junit.jupiter.api.Test; @@ -106,9 +106,7 @@ public void testSuspendAndRecoverAfterRescaleOverwrite() throws Exception { assertThat(batchSql("SELECT * FROM T3")).containsExactlyInAnyOrderElementsOf(committedData); // step5: resume streaming job - sEnv.getConfig() - .getConfiguration() - .set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + sEnv.getConfig().getConfiguration().set(StateRecoveryOptions.SAVEPOINT_PATH, savepointPath); JobClient resumedJobClient = startJobAndCommitSnapshot(streamSql, snapshotAfterRescale.id()); // stop job diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java index f6dfb1b23046..eeb99df75cb5 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java @@ -50,7 +50,7 @@ import java.util.List; import java.util.Random; -import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL; +import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTING_INTERVAL; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java index 6b912d2e57fe..b1486deacb0c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java @@ -137,7 +137,7 @@ private JobClient runRecoverFromSavepointJob(String failingPath, String savepoin .parallelism(1) .allowRestart() .setConf(conf) - .setConf(StateBackendOptions.STATE_BACKEND, "filesystem") + .setConf(StateBackendOptions.STATE_BACKEND, "hashmap") .setConf( CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + path + "/checkpoint") diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java index ce0017eb1874..1d4d6c8be3b6 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java @@ -23,13 +23,13 @@ import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.dag.Transformation; import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -183,7 +183,7 @@ public TableEnvironment build() { tEnv.getConfig() .getConfiguration() .set( - ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, + CheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofMillis(checkpointIntervalMs)); } } else { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java index 86b0014eb39c..9c3170f9a96b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java @@ -23,8 +23,9 @@ import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -75,12 +76,11 @@ public static void init(String warehouse) { } public static void init(String warehouse, int parallelism) { - StreamExecutionEnvironment sExeEnv = buildStreamEnv(parallelism); - sExeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + // Using `none` to avoid compatibility issues with Flink 1.18-. + StreamExecutionEnvironment sExeEnv = buildStreamEnv(parallelism, "none"); sEnv = StreamTableEnvironment.create(sExeEnv); - bExeEnv = buildBatchEnv(parallelism); - bExeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + bExeEnv = buildBatchEnv(parallelism, "none"); bEnv = StreamTableEnvironment.create(bExeEnv, EnvironmentSettings.inBatchMode()); ReadWriteTableTestUtil.warehouse = warehouse; @@ -95,16 +95,24 @@ public static void init(String warehouse, int parallelism) { bEnv.useCatalog(catalog); } - public static StreamExecutionEnvironment buildStreamEnv(int parallelism) { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + public static StreamExecutionEnvironment buildStreamEnv( + int parallelism, String restartStrategy) { + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, restartStrategy); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); env.setRuntimeMode(RuntimeExecutionMode.STREAMING); env.enableCheckpointing(100); env.setParallelism(parallelism); return env; } - public static StreamExecutionEnvironment buildBatchEnv(int parallelism) { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + public static StreamExecutionEnvironment buildBatchEnv( + int parallelism, String restartStrategy) { + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, restartStrategy); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); env.setRuntimeMode(RuntimeExecutionMode.BATCH); env.setParallelism(parallelism); return env; diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java index 74d2d7e1c343..d30e7f83482f 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java @@ -34,9 +34,9 @@ import com.klarna.hiverunner.HiveShell; import com.klarna.hiverunner.annotations.HiveSQL; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; @@ -139,7 +139,7 @@ private void registerHiveCatalog(String catalogName, Map catalog EnvironmentSettings.newInstance().inStreamingMode().build()); sEnv.getConfig() .getConfiguration() - .set(ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofSeconds(1)); + .set(CheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofSeconds(1)); sEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); tEnv.executeSql( From 346911dd1e066e6c1ef6006a298a6eb90d9650f2 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Sat, 23 Nov 2024 10:49:41 +0800 Subject: [PATCH 06/17] [flink] Avoid deprecated DataStreamUtils --- .../org/apache/paimon/flink/source/IteratorSourcesITCase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java index 8404d994fa9f..0c5d485af7bc 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java @@ -18,10 +18,10 @@ package org.apache.paimon.flink.source; +import org.apache.commons.collections.IteratorUtils; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamUtils; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; import org.apache.flink.test.util.MiniClusterWithClientResource; @@ -67,7 +67,7 @@ public void testParallelSourceExecution() throws Exception { "iterator source"); final List result = - DataStreamUtils.collectBoundedStream(stream, "Iterator Source Test"); + IteratorUtils.toList(stream.executeAndCollect("Iterator Source Test")); verifySequence(result, 1L, 1_000L); } From e1f2c682afd513ee5c49d0faa571cab7bd12a2aa Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Sat, 23 Nov 2024 10:53:15 +0800 Subject: [PATCH 07/17] [flink] Remove deprecated TestEnvironment --- .../paimon/flink/util/MiniClusterWithClientExtension.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java index cfc23a0a44d8..39939f78670b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.junit5.InjectClusterClient; -import org.apache.flink.test.util.TestEnvironment; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.AfterEachCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; @@ -167,17 +166,12 @@ private void registerEnv(InternalMiniClusterExtension internalMiniClusterExtensi .getOptional(CoreOptions.DEFAULT_PARALLELISM) .orElse(internalMiniClusterExtension.getNumberSlots()); - TestEnvironment executionEnvironment = - new TestEnvironment( - internalMiniClusterExtension.getMiniCluster(), defaultParallelism, false); - executionEnvironment.setAsContext(); TestStreamEnvironment.setAsContext( internalMiniClusterExtension.getMiniCluster(), defaultParallelism); } private void unregisterEnv(InternalMiniClusterExtension internalMiniClusterExtension) { TestStreamEnvironment.unsetAsContext(); - TestEnvironment.unsetAsContext(); } private MiniClusterClient createMiniClusterClient( From 5a6db981caa4f54f265cd7f343d8fd4c052e817c Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Sat, 23 Nov 2024 10:55:48 +0800 Subject: [PATCH 08/17] [flink] Replace deprecated flink Time with java Duration --- .../apache/paimon/flink/util/ReadWriteTableTestUtil.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java index 9c3170f9a96b..0eac2ed2936e 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java @@ -23,7 +23,6 @@ import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -37,6 +36,7 @@ import javax.annotation.Nullable; import java.nio.file.Paths; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -53,7 +53,7 @@ /** Test util for {@link ReadWriteTableITCase}. */ public class ReadWriteTableTestUtil { - private static final Time TIME_OUT = Time.seconds(10); + private static final Duration TIME_OUT = Duration.ofSeconds(10); public static final int DEFAULT_PARALLELISM = 2; @@ -278,7 +278,7 @@ public static void testBatchRead(String query, List expected) throws Except try (BlockingIterator iterator = BlockingIterator.of(resultItr)) { if (!expected.isEmpty()) { List result = - iterator.collect(expected.size(), TIME_OUT.getSize(), TIME_OUT.getUnit()); + iterator.collect(expected.size(), TIME_OUT.getSeconds(), TimeUnit.SECONDS); assertThat(toInsertOnlyRows(result)) .containsExactlyInAnyOrderElementsOf(toInsertOnlyRows(expected)); } From 60b51e68bfdda83dc10ecebeb1e1800fa3caf5e5 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:34:37 +0800 Subject: [PATCH 09/17] [flink] Avoid deprecated SingleThreadMultiplexSourceReaderBase constructor --- .../flink/source/FileStoreSourceReader.java | 23 ------------------- .../source/align/AlignedSourceReader.java | 2 +- 2 files changed, 1 insertion(+), 24 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java index 92adf5e04998..8fc78c868ba5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java @@ -25,9 +25,7 @@ import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.file.src.reader.BulkFormat.RecordIterator; import org.apache.flink.table.data.RowData; @@ -64,27 +62,6 @@ public FileStoreSourceReader( this.ioManager = ioManager; } - public FileStoreSourceReader( - SourceReaderContext readerContext, - TableRead tableRead, - FileStoreSourceReaderMetrics metrics, - IOManager ioManager, - @Nullable Long limit, - FutureCompletingBlockingQueue>> - elementsQueue) { - super( - elementsQueue, - () -> - new FileStoreSourceSplitReader( - tableRead, RecordLimiter.create(limit), metrics), - (element, output, state) -> - FlinkRecordsWithSplitIds.emitRecord( - readerContext, element, output, state, metrics), - readerContext.getConfiguration(), - readerContext); - this.ioManager = ioManager; - } - @Override public void start() { // we request a split only if we did not get splits during the checkpoint restore diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java index 1f0bbca314b6..a8ffe3de561f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java @@ -58,7 +58,7 @@ public AlignedSourceReader( @Nullable Long limit, FutureCompletingBlockingQueue>> elementsQueue) { - super(readerContext, tableRead, metrics, ioManager, limit, elementsQueue); + super(readerContext, tableRead, metrics, ioManager, limit); this.elementsQueue = elementsQueue; this.nextCheckpointId = null; } From 23e63b6be7042e751f30b1f6f7218612909d1eef Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:35:14 +0800 Subject: [PATCH 10/17] [flink] Avoid deprecated FileSystem#getKind --- .../org/apache/paimon/flink/FlinkFileIO.java | 24 +++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java index 74512409bfc8..617d25125f37 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java @@ -29,10 +29,10 @@ import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystem.WriteMode; -import org.apache.flink.core.fs.FileSystemKind; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Locale; /** Flink {@link FileIO} to use {@link FileSystem}. */ public class FlinkFileIO implements FileIO { @@ -48,7 +48,27 @@ public FlinkFileIO(Path path) { @Override public boolean isObjectStore() { try { - return path.getFileSystem().getKind() != FileSystemKind.FILE_SYSTEM; + FileSystem fs = path.getFileSystem(); + String scheme = fs.getUri().getScheme().toLowerCase(Locale.US); + + if (scheme.startsWith("s3") + || scheme.startsWith("emr") + || scheme.startsWith("oss") + || scheme.startsWith("wasb") + || scheme.startsWith("gs")) { + // the Amazon S3 storage or Aliyun OSS storage or Azure Blob Storage + // or Google Cloud Storage + return true; + } else if (scheme.startsWith("http") || scheme.startsWith("ftp")) { + // file servers instead of file systems + // they might actually be consistent, but we have no hard guarantees + // currently to rely on that + return true; + } else { + // the remainder should include hdfs, kosmos, ceph, ... + // this also includes federated HDFS (viewfs). + return false; + } } catch (IOException e) { throw new UncheckedIOException(e); } From e402f16f39508f7f0ca5c54ba3e814640f9db7b9 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:35:45 +0800 Subject: [PATCH 11/17] [flink] Avoid deprecated SetupableStreamOperator --- .../sink/cdc/FlinkCdcMultiTableSink.java | 5 +- .../AppendBypassCompactWorkerOperator.java | 2 - .../AutoTagForSavepointCommitterOperator.java | 24 +--- ...gForSavepointCommitterOperatorFactory.java | 94 ++++++++++++++ .../sink/BatchWriteGeneratorTagOperator.java | 24 +--- ...BatchWriteGeneratorTagOperatorFactory.java | 60 +++++++++ .../sink/CombinedTableCompactorSink.java | 6 +- .../paimon/flink/sink/CommitterOperator.java | 26 +--- .../flink/sink/CommitterOperatorFactory.java | 115 ++++++++++++++++++ .../apache/paimon/flink/sink/FlinkSink.java | 19 +-- .../paimon/flink/sink/LocalMergeOperator.java | 2 - .../flink/sink/PrepareCommitOperator.java | 2 - .../paimon/flink/sink/UnawareBucketSink.java | 2 + .../sink/index/IndexBootstrapOperator.java | 2 - .../AppendBypassCoordinateOperator.java | 8 +- ...AppendBypassCoordinateOperatorFactory.java | 6 +- ...oTagForSavepointCommitterOperatorTest.java | 14 +-- .../BatchWriteGeneratorTagOperatorTest.java | 27 +++- .../flink/sink/CommitterOperatorTest.java | 64 ++++++---- .../flink/sink/StoreMultiCommitterTest.java | 13 +- 20 files changed, 379 insertions(+), 136 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index 55e987c6055f..c13928ab164e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -21,7 +21,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.sink.CommittableStateManager; import org.apache.paimon.flink.sink.Committer; -import org.apache.paimon.flink.sink.CommitterOperator; +import org.apache.paimon.flink.sink.CommitterOperatorFactory; import org.apache.paimon.flink.sink.FlinkSink; import org.apache.paimon.flink.sink.FlinkStreamPartitioner; import org.apache.paimon.flink.sink.MultiTableCommittable; @@ -129,10 +129,9 @@ public DataStreamSink sinkFrom( .transform( GLOBAL_COMMITTER_NAME, typeInfo, - new CommitterOperator<>( + new CommitterOperatorFactory<>( true, false, - commitChaining, commitUser, createCommitterFactory(), createCommittableStateManager())) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java index 92cd31ea8aa2..7c9ba436afb3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java @@ -21,7 +21,6 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.types.Either; @@ -31,7 +30,6 @@ public class AppendBypassCompactWorkerOperator public AppendBypassCompactWorkerOperator(FileStoreTable table, String commitUser) { super(table, commitUser); - this.chainingStrategy = ChainingStrategy.HEAD; } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java index 6d27c6019483..0822f0461241 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java @@ -32,18 +32,13 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointStreamFactory; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; -import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.SetupableStreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import java.time.Duration; @@ -58,9 +53,7 @@ * time, tags are automatically created for each flink savepoint. */ public class AutoTagForSavepointCommitterOperator - implements OneInputStreamOperator, - SetupableStreamOperator, - BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { public static final String SAVEPOINT_TAG_PREFIX = "savepoint-"; private static final long serialVersionUID = 1L; @@ -256,19 +249,4 @@ public void setKeyContextElement(StreamRecord record) throws Exception public void endInput() throws Exception { commitOperator.endInput(); } - - @Override - public void setup(StreamTask containingTask, StreamConfig config, Output output) { - commitOperator.setup(containingTask, config, output); - } - - @Override - public ChainingStrategy getChainingStrategy() { - return commitOperator.getChainingStrategy(); - } - - @Override - public void setChainingStrategy(ChainingStrategy strategy) { - commitOperator.setChainingStrategy(strategy); - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java new file mode 100644 index 000000000000..1787f8e7adce --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java @@ -0,0 +1,94 @@ +/* + * 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; + +import org.apache.paimon.operation.TagDeletion; +import org.apache.paimon.table.sink.TagCallback; +import org.apache.paimon.utils.SerializableSupplier; +import org.apache.paimon.utils.SnapshotManager; +import org.apache.paimon.utils.TagManager; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + +import java.time.Duration; +import java.util.List; +import java.util.NavigableSet; +import java.util.TreeSet; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * AutoTagForSavepointCommitterOperator}. + */ +public class AutoTagForSavepointCommitterOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + + private final CommitterOperatorFactory commitOperatorFactory; + + private final SerializableSupplier snapshotManagerFactory; + + private final SerializableSupplier tagManagerFactory; + + private final SerializableSupplier tagDeletionFactory; + + private final SerializableSupplier> callbacksSupplier; + + private final NavigableSet identifiersForTags; + + private final Duration tagTimeRetained; + + public AutoTagForSavepointCommitterOperatorFactory( + CommitterOperatorFactory commitOperatorFactory, + SerializableSupplier snapshotManagerFactory, + SerializableSupplier tagManagerFactory, + SerializableSupplier tagDeletionFactory, + SerializableSupplier> callbacksSupplier, + Duration tagTimeRetained) { + this.commitOperatorFactory = commitOperatorFactory; + this.tagManagerFactory = tagManagerFactory; + this.snapshotManagerFactory = snapshotManagerFactory; + this.tagDeletionFactory = tagDeletionFactory; + this.callbacksSupplier = callbacksSupplier; + this.identifiersForTags = new TreeSet<>(); + this.tagTimeRetained = tagTimeRetained; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new AutoTagForSavepointCommitterOperator<>( + commitOperatorFactory.createStreamOperator(parameters), + snapshotManagerFactory, + tagManagerFactory, + tagDeletionFactory, + callbacksSupplier, + tagTimeRetained); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AutoTagForSavepointCommitterOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java index 23202b45077f..1cbcc4b2262f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java @@ -28,18 +28,13 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointStreamFactory; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; -import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.SetupableStreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import java.time.Instant; @@ -53,9 +48,7 @@ * completed, the corresponding tag is generated. */ public class BatchWriteGeneratorTagOperator - implements OneInputStreamOperator, - SetupableStreamOperator, - BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { private static final String BATCH_WRITE_TAG_PREFIX = "batch-write-"; @@ -250,19 +243,4 @@ public void setKeyContextElement(StreamRecord record) throws Exception public void endInput() throws Exception { commitOperator.endInput(); } - - @Override - public void setup(StreamTask containingTask, StreamConfig config, Output output) { - commitOperator.setup(containingTask, config, output); - } - - @Override - public ChainingStrategy getChainingStrategy() { - return commitOperator.getChainingStrategy(); - } - - @Override - public void setChainingStrategy(ChainingStrategy strategy) { - commitOperator.setChainingStrategy(strategy); - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java new file mode 100644 index 000000000000..e3c0e5c49168 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java @@ -0,0 +1,60 @@ +/* + * 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; + +import org.apache.paimon.table.FileStoreTable; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * BatchWriteGeneratorTagOperator}. + */ +public class BatchWriteGeneratorTagOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final CommitterOperatorFactory commitOperatorFactory; + + protected final FileStoreTable table; + + public BatchWriteGeneratorTagOperatorFactory( + CommitterOperatorFactory commitOperatorFactory, + FileStoreTable table) { + this.table = table; + this.commitOperatorFactory = commitOperatorFactory; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new BatchWriteGeneratorTagOperator<>( + commitOperatorFactory.createStreamOperator(parameters), table); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return BatchWriteGeneratorTagOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java index 87a28091fa30..fc84dd671469 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java @@ -153,15 +153,17 @@ protected DataStreamSink doCommit( .transform( GLOBAL_COMMITTER_NAME, new MultiTableCommittableTypeInfo(), - new CommitterOperator<>( + new CommitterOperatorFactory<>( streamingCheckpointEnabled, false, - options.get(SINK_COMMITTER_OPERATOR_CHAINING), commitUser, createCommitterFactory(isStreaming), createCommittableStateManager(), options.get(END_INPUT_WATERMARK))) .setParallelism(written.getParallelism()); + if (!options.get(SINK_COMMITTER_OPERATOR_CHAINING)) { + committed = committed.startNewChain(); + } return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java index 021a5db413d5..383cbcd6ebf7 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java @@ -25,8 +25,8 @@ import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -91,26 +91,9 @@ public class CommitterOperator extends AbstractStreamOpe private final Long endInputWatermark; public CommitterOperator( + StreamOperatorParameters parameters, boolean streamingCheckpointEnabled, boolean forceSingleParallelism, - boolean chaining, - String initialCommitUser, - Committer.Factory committerFactory, - CommittableStateManager committableStateManager) { - this( - streamingCheckpointEnabled, - forceSingleParallelism, - chaining, - initialCommitUser, - committerFactory, - committableStateManager, - null); - } - - public CommitterOperator( - boolean streamingCheckpointEnabled, - boolean forceSingleParallelism, - boolean chaining, String initialCommitUser, Committer.Factory committerFactory, CommittableStateManager committableStateManager, @@ -122,7 +105,10 @@ public CommitterOperator( this.committerFactory = checkNotNull(committerFactory); this.committableStateManager = committableStateManager; this.endInputWatermark = endInputWatermark; - setChainingStrategy(chaining ? ChainingStrategy.ALWAYS : ChainingStrategy.HEAD); + this.setup( + parameters.getContainingTask(), + parameters.getStreamConfig(), + parameters.getOutput()); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java new file mode 100644 index 000000000000..cce3d4e176bf --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java @@ -0,0 +1,115 @@ +/* + * 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; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + +import java.util.NavigableMap; +import java.util.TreeMap; + +import static org.apache.paimon.utils.Preconditions.checkNotNull; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * CommitterOperator}. + */ +public class CommitterOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + protected final boolean streamingCheckpointEnabled; + + /** Whether to check the parallelism while runtime. */ + protected final boolean forceSingleParallelism; + /** + * This commitUser is valid only for new jobs. After the job starts, this commitUser will be + * recorded into the states of write and commit operators. When the job restarts, commitUser + * will be recovered from states and this value is ignored. + */ + protected final String initialCommitUser; + + /** Group the committable by the checkpoint id. */ + protected final NavigableMap committablesPerCheckpoint; + + protected final Committer.Factory committerFactory; + + protected final CommittableStateManager committableStateManager; + + /** + * Aggregate committables to global committables and commit the global committables to the + * external system. + */ + protected Committer committer; + + protected final Long endInputWatermark; + + public CommitterOperatorFactory( + boolean streamingCheckpointEnabled, + boolean forceSingleParallelism, + String initialCommitUser, + Committer.Factory committerFactory, + CommittableStateManager committableStateManager) { + this( + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + null); + } + + public CommitterOperatorFactory( + boolean streamingCheckpointEnabled, + boolean forceSingleParallelism, + String initialCommitUser, + Committer.Factory committerFactory, + CommittableStateManager committableStateManager, + Long endInputWatermark) { + this.streamingCheckpointEnabled = streamingCheckpointEnabled; + this.forceSingleParallelism = forceSingleParallelism; + this.initialCommitUser = initialCommitUser; + this.committablesPerCheckpoint = new TreeMap<>(); + this.committerFactory = checkNotNull(committerFactory); + this.committableStateManager = committableStateManager; + this.endInputWatermark = endInputWatermark; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CommitterOperator<>( + parameters, + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + endInputWatermark); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CommitterOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index 28858b85a4a7..214e2858ae53 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -46,6 +46,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import javax.annotation.Nullable; @@ -269,11 +270,10 @@ protected DataStreamSink doCommit(DataStream written, String com } Options options = Options.fromMap(table.options()); - OneInputStreamOperator committerOperator = - new CommitterOperator<>( + OneInputStreamOperatorFactory committerOperator = + new CommitterOperatorFactory<>( streamingCheckpointEnabled, true, - options.get(SINK_COMMITTER_OPERATOR_CHAINING), commitUser, createCommitterFactory(), createCommittableStateManager(), @@ -281,8 +281,9 @@ protected DataStreamSink doCommit(DataStream written, String com if (options.get(SINK_AUTO_TAG_FOR_SAVEPOINT)) { committerOperator = - new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) committerOperator, + new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) + committerOperator, table::snapshotManager, table::tagManager, () -> table.store().newTagDeletion(), @@ -292,8 +293,9 @@ protected DataStreamSink doCommit(DataStream written, String com if (conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.BATCH && table.coreOptions().tagCreationMode() == TagCreationMode.BATCH) { committerOperator = - new BatchWriteGeneratorTagOperator<>( - (CommitterOperator) committerOperator, + new BatchWriteGeneratorTagOperatorFactory<>( + (CommitterOperatorFactory) + committerOperator, table); } SingleOutputStreamOperator committed = @@ -311,6 +313,9 @@ protected DataStreamSink doCommit(DataStream written, String com table.name(), options.get(SINK_OPERATOR_UID_SUFFIX))); } + if (!options.get(SINK_COMMITTER_OPERATOR_CHAINING)) { + committed = committed.startNewChain(); + } configureGlobalCommitter( committed, options.get(SINK_COMMITTER_CPU), options.get(SINK_COMMITTER_MEMORY)); return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java index 6931fe907218..e38844f60a73 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java @@ -45,7 +45,6 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.watermark.Watermark; @@ -82,7 +81,6 @@ public LocalMergeOperator(TableSchema schema) { "LocalMergeOperator currently only support tables with primary keys"); this.schema = schema; this.ignoreDelete = CoreOptions.fromMap(schema.options()).ignoreDelete(); - setChainingStrategy(ChainingStrategy.ALWAYS); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java index 3668386ddc2d..9e82ffd9ff41 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java @@ -27,7 +27,6 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -54,7 +53,6 @@ public abstract class PrepareCommitOperator extends AbstractStreamOpera public PrepareCommitOperator(Options options) { this.options = options; - setChainingStrategy(ChainingStrategy.ALWAYS); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java index 98b58aa8e96d..f59e728a70f4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java @@ -74,11 +74,13 @@ public DataStream doWrite( new CommittableTypeInfo(), new CompactionTaskTypeInfo()), new AppendBypassCoordinateOperatorFactory<>(table)) + .startNewChain() .forceNonParallel() .transform( "Compact Worker: " + table.name(), new CommittableTypeInfo(), new AppendBypassCompactWorkerOperator(table, initialCommitUser)) + .startNewChain() .setParallelism(written.getParallelism()); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java index 5c8ba8f9441f..3eb1f11a02a5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java @@ -27,7 +27,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -44,7 +43,6 @@ public IndexBootstrapOperator( IndexBootstrap bootstrap, SerializableFunction converter) { this.bootstrap = bootstrap; this.converter = converter; - setChainingStrategy(ChainingStrategy.ALWAYS); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java index 45090f7b68b4..b8b0d61e10a9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java @@ -26,8 +26,8 @@ import org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.types.Either; @@ -58,10 +58,12 @@ public class AppendBypassCoordinateOperator private transient LinkedBlockingQueue compactTasks; public AppendBypassCoordinateOperator( - FileStoreTable table, ProcessingTimeService processingTimeService) { + StreamOperatorParameters> parameters, + FileStoreTable table, + ProcessingTimeService processingTimeService) { this.table = table; this.processingTimeService = processingTimeService; - this.chainingStrategy = ChainingStrategy.HEAD; + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java index 7c53e01b47e6..a4c51e5b5a9b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java @@ -45,11 +45,7 @@ T createStreamOperator( StreamOperatorParameters> parameters) { AppendBypassCoordinateOperator operator = - new AppendBypassCoordinateOperator<>(table, processingTimeService); - operator.setup( - parameters.getContainingTask(), - parameters.getStreamConfig(), - parameters.getOutput()); + new AppendBypassCoordinateOperator<>(parameters, table, processingTimeService); return (T) operator; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java index 3b58c24d16b1..1d3b0b37a999 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.SavepointType; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.jupiter.api.Test; @@ -198,12 +198,12 @@ private void processCommittable( } @Override - protected OneInputStreamOperator createCommitterOperator( + protected OneInputStreamOperatorFactory createCommitterOperator( FileStoreTable table, String commitUser, CommittableStateManager committableStateManager) { - return new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) + return new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) super.createCommitterOperator(table, commitUser, committableStateManager), table::snapshotManager, table::tagManager, @@ -213,13 +213,13 @@ protected OneInputStreamOperator createCommitterOperat } @Override - protected OneInputStreamOperator createCommitterOperator( + protected OneInputStreamOperatorFactory createCommitterOperator( FileStoreTable table, String commitUser, CommittableStateManager committableStateManager, ThrowingConsumer initializeFunction) { - return new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) + return new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) super.createCommitterOperator( table, commitUser, committableStateManager, initializeFunction), table::snapshotManager, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java index 147110637aef..7fef39a6e20f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java @@ -27,13 +27,21 @@ import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.junit.jupiter.api.Test; import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; import java.time.format.DateTimeFormatter; +import java.util.ArrayList; import java.util.HashMap; import java.util.Objects; @@ -54,12 +62,23 @@ public void testBatchWriteGeneratorTag() throws Exception { StreamTableWrite write = table.newStreamWriteBuilder().withCommitUser(initialCommitUser).newWrite(); - OneInputStreamOperator committerOperator = + OneInputStreamOperatorFactory committerOperatorFactory = createCommitterOperator( table, initialCommitUser, new RestoreAndFailCommittableStateManager<>( ManifestCommittableSerializer::new)); + + OneInputStreamOperator committerOperator = + committerOperatorFactory.createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask(new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); + committerOperator.open(); TableCommitImpl tableCommit = table.newCommit(initialCommitUser); @@ -106,12 +125,12 @@ public void testBatchWriteGeneratorTag() throws Exception { } @Override - protected OneInputStreamOperator createCommitterOperator( + protected OneInputStreamOperatorFactory createCommitterOperator( FileStoreTable table, String commitUser, CommittableStateManager committableStateManager) { - return new BatchWriteGeneratorTagOperator<>( - (CommitterOperator) + return new BatchWriteGeneratorTagOperatorFactory<>( + (CommitterOperatorFactory) super.createCommitterOperator(table, commitUser, committableStateManager), table); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java index 668d651236fd..3c77fef8b7fe 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java @@ -51,10 +51,13 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.util.Preconditions; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -259,7 +262,7 @@ public void testRestoreCommitUser() throws Exception { // 3. Check whether success List actual = new ArrayList<>(); - OneInputStreamOperator operator = + OneInputStreamOperatorFactory operator = createCommitterOperator( table, initialCommitUser, @@ -315,7 +318,7 @@ public void testRestoreEmptyMarkDoneState() throws Exception { public void testCommitInputEnd() throws Exception { FileStoreTable table = createFileStoreTable(); String commitUser = UUID.randomUUID().toString(); - OneInputStreamOperator operator = + OneInputStreamOperatorFactory operator = createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -604,7 +607,7 @@ public void testCalcDataBytesSend() throws Exception { public void testCommitMetrics() throws Exception { FileStoreTable table = createFileStoreTable(); - OneInputStreamOperator operator = + OneInputStreamOperatorFactory operator = createCommitterOperator( table, null, @@ -627,7 +630,9 @@ public void testCommitMetrics() throws Exception { testHarness.notifyOfCompletedCheckpoint(cpId); MetricGroup commitMetricGroup = - operator.getMetricGroup() + testHarness + .getOneInputOperator() + .getMetricGroup() .addGroup("paimon") .addGroup("table", table.name()) .addGroup("commit"); @@ -685,7 +690,7 @@ public void testCommitMetrics() throws Exception { public void testParallelism() throws Exception { FileStoreTable table = createFileStoreTable(); String commitUser = UUID.randomUUID().toString(); - OneInputStreamOperator operator = + OneInputStreamOperatorFactory operator = createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); try (OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator, 10, 10, 3)) { @@ -700,7 +705,7 @@ public void testParallelism() throws Exception { protected OneInputStreamOperatorTestHarness createRecoverableTestHarness(FileStoreTable table) throws Exception { - OneInputStreamOperator operator = + OneInputStreamOperatorFactory operator = createCommitterOperator( table, null, @@ -716,18 +721,18 @@ private OneInputStreamOperatorTestHarness createLossyT private OneInputStreamOperatorTestHarness createLossyTestHarness( FileStoreTable table, String commitUser) throws Exception { - OneInputStreamOperator operator = + OneInputStreamOperatorFactory operator = createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); return createTestHarness(operator); } private OneInputStreamOperatorTestHarness createTestHarness( - OneInputStreamOperator operator) throws Exception { + OneInputStreamOperatorFactory operator) throws Exception { return createTestHarness(operator, 1, 1, 0); } private OneInputStreamOperatorTestHarness createTestHarness( - OneInputStreamOperator operator, + OneInputStreamOperatorFactory operator, int maxParallelism, int parallelism, int subTaskIndex) @@ -736,22 +741,18 @@ private OneInputStreamOperatorTestHarness createTestHa new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>( - operator, - maxParallelism, - parallelism, - subTaskIndex, - serializer, - new OperatorID()); + operator, maxParallelism, parallelism, subTaskIndex, new OperatorID()); + harness.getStreamConfig().setupNetworkInputs(Preconditions.checkNotNull(serializer)); + harness.getStreamConfig().serializeAllConfigs(); harness.setup(serializer); return harness; } - protected OneInputStreamOperator createCommitterOperator( + protected OneInputStreamOperatorFactory createCommitterOperator( FileStoreTable table, String commitUser, CommittableStateManager committableStateManager) { - return new CommitterOperator<>( - true, + return new CommitterOperatorFactory<>( true, true, commitUser == null ? initialCommitUser : commitUser, @@ -765,13 +766,12 @@ protected OneInputStreamOperator createCommitterOperat committableStateManager); } - protected OneInputStreamOperator createCommitterOperator( + protected OneInputStreamOperatorFactory createCommitterOperator( FileStoreTable table, String commitUser, CommittableStateManager committableStateManager, ThrowingConsumer initializeFunction) { - return new CommitterOperator( - true, + return new CommitterOperatorFactory( true, true, commitUser == null ? initialCommitUser : commitUser, @@ -784,8 +784,24 @@ protected OneInputStreamOperator createCommitterOperat context), committableStateManager) { @Override - public void initializeState(StateInitializationContext context) throws Exception { - initializeFunction.accept(context); + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CommitterOperator( + parameters, + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + endInputWatermark) { + @Override + public void initializeState(StateInitializationContext context) + throws Exception { + initializeFunction.accept(context); + } + }; } }; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java index 10e432f3c8c2..f60397fadd4c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java @@ -645,11 +645,10 @@ public void testCommitMetrics() throws Exception { private OneInputStreamOperatorTestHarness createRecoverableTestHarness() throws Exception { - CommitterOperator operator = - new CommitterOperator<>( + CommitterOperatorFactory operator = + new CommitterOperatorFactory<>( true, false, - true, initialCommitUser, context -> new StoreMultiCommitter(catalogLoader, context), new RestoreAndFailCommittableStateManager<>( @@ -659,11 +658,10 @@ public void testCommitMetrics() throws Exception { private OneInputStreamOperatorTestHarness createLossyTestHarness() throws Exception { - CommitterOperator operator = - new CommitterOperator<>( + CommitterOperatorFactory operator = + new CommitterOperatorFactory<>( true, false, - true, initialCommitUser, context -> new StoreMultiCommitter(catalogLoader, context), new CommittableStateManager() { @@ -682,7 +680,8 @@ public void snapshotState( private OneInputStreamOperatorTestHarness createTestHarness( - CommitterOperator operator) + CommitterOperatorFactory + operator) throws Exception { TypeSerializer serializer = new MultiTableCommittableTypeInfo().createSerializer(new ExecutionConfig()); From 51052d1864c853e6e38c9c70df4af959931cbe7d Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:36:16 +0800 Subject: [PATCH 12/17] [flink] Avoid deprecated usage on TableSchema, DataType and DescriptorProperties --- .../apache/paimon/flink/DataCatalogTable.java | 115 ++++++++++++----- .../org/apache/paimon/flink/FlinkCatalog.java | 55 ++++---- .../paimon/flink/FlinkGenericCatalog.java | 6 - .../paimon/flink/SystemCatalogTable.java | 12 +- .../utils/FlinkCatalogPropertiesUtil.java | 88 ++++++------- .../utils/FlinkDescriptorProperties.java | 99 +++++++++++++++ .../flink/FlinkCatalogPropertiesUtilTest.java | 117 +++++++++++++----- .../apache/paimon/flink/FlinkCatalogTest.java | 9 +- 8 files changed, 353 insertions(+), 148 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java index 019d7bd6892f..25431c1c688a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java @@ -23,33 +23,55 @@ import org.apache.paimon.types.DataField; import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.constraints.UniqueConstraint; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; -/** A {@link CatalogTableImpl} to wrap {@link FileStoreTable}. */ -public class DataCatalogTable extends CatalogTableImpl { +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A {@link CatalogTable} to wrap {@link FileStoreTable}. */ +public class DataCatalogTable implements CatalogTable { + // Schema of the table (column names and types) + private final Schema schema; + + // Partition keys if this is a partitioned table. It's an empty set if the table is not + // partitioned + private final List partitionKeys; + + // Properties of the table + private final Map options; + + // Comment of the table + private final String comment; private final Table table; private final Map nonPhysicalColumnComments; public DataCatalogTable( Table table, - TableSchema tableSchema, + Schema resolvedSchema, List partitionKeys, - Map properties, + Map options, String comment, Map nonPhysicalColumnComments) { - super(tableSchema, partitionKeys, properties, comment); + this.schema = resolvedSchema; + this.partitionKeys = checkNotNull(partitionKeys, "partitionKeys cannot be null"); + this.options = checkNotNull(options, "options cannot be null"); + + checkArgument( + options.entrySet().stream() + .allMatch(e -> e.getKey() != null && e.getValue() != null), + "properties cannot have null keys or values"); + + this.comment = comment; + this.table = table; this.nonPhysicalColumnComments = nonPhysicalColumnComments; } @@ -66,32 +88,30 @@ public Schema getUnresolvedSchema() { .filter(dataField -> dataField.description() != null) .collect(Collectors.toMap(DataField::name, DataField::description)); - return toSchema(getSchema(), columnComments); + return toSchema(schema, columnComments); } - /** Copied from {@link TableSchema#toSchema(Map)} to support versions lower than 1.17. */ - private Schema toSchema(TableSchema tableSchema, Map comments) { + private Schema toSchema(Schema tableSchema, Map comments) { final Schema.Builder builder = Schema.newBuilder(); - tableSchema - .getTableColumns() + .getColumns() .forEach( column -> { - if (column instanceof TableColumn.PhysicalColumn) { - final TableColumn.PhysicalColumn c = - (TableColumn.PhysicalColumn) column; - builder.column(c.getName(), c.getType()); - } else if (column instanceof TableColumn.MetadataColumn) { - final TableColumn.MetadataColumn c = - (TableColumn.MetadataColumn) column; + if (column instanceof Schema.UnresolvedPhysicalColumn) { + final Schema.UnresolvedPhysicalColumn c = + (Schema.UnresolvedPhysicalColumn) column; + builder.column(c.getName(), c.getDataType()); + } else if (column instanceof Schema.UnresolvedMetadataColumn) { + final Schema.UnresolvedMetadataColumn c = + (Schema.UnresolvedMetadataColumn) column; builder.columnByMetadata( c.getName(), - c.getType(), - c.getMetadataAlias().orElse(null), + c.getDataType(), + c.getMetadataKey(), c.isVirtual()); - } else if (column instanceof TableColumn.ComputedColumn) { - final TableColumn.ComputedColumn c = - (TableColumn.ComputedColumn) column; + } else if (column instanceof Schema.UnresolvedComputedColumn) { + final Schema.UnresolvedComputedColumn c = + (Schema.UnresolvedComputedColumn) column; builder.columnByExpression(c.getName(), c.getExpression()); } else { throw new IllegalArgumentException( @@ -104,19 +124,16 @@ private Schema toSchema(TableSchema tableSchema, Map comments) { builder.withComment(nonPhysicalColumnComments.get(colName)); } }); - tableSchema .getWatermarkSpecs() .forEach( spec -> builder.watermark( - spec.getRowtimeAttribute(), spec.getWatermarkExpr())); - + spec.getColumnName(), spec.getWatermarkExpression())); if (tableSchema.getPrimaryKey().isPresent()) { - UniqueConstraint primaryKey = tableSchema.getPrimaryKey().get(); - builder.primaryKeyNamed(primaryKey.getName(), primaryKey.getColumns()); + Schema.UnresolvedPrimaryKey primaryKey = tableSchema.getPrimaryKey().get(); + builder.primaryKeyNamed(primaryKey.getConstraintName(), primaryKey.getColumnNames()); } - return builder.build(); } @@ -124,7 +141,7 @@ private Schema toSchema(TableSchema tableSchema, Map comments) { public CatalogBaseTable copy() { return new DataCatalogTable( table, - getSchema().copy(), + Schema.newBuilder().fromSchema(schema).build(), new ArrayList<>(getPartitionKeys()), new HashMap<>(getOptions()), getComment(), @@ -135,10 +152,40 @@ public CatalogBaseTable copy() { public CatalogTable copy(Map options) { return new DataCatalogTable( table, - getSchema(), + Schema.newBuilder().fromSchema(schema).build(), getPartitionKeys(), options, getComment(), nonPhysicalColumnComments); } + + @Override + public Optional getDescription() { + return Optional.of(getComment()); + } + + @Override + public Optional getDetailedDescription() { + return Optional.of("This is a catalog table in an im-memory catalog"); + } + + @Override + public boolean isPartitioned() { + return !partitionKeys.isEmpty(); + } + + @Override + public List getPartitionKeys() { + return partitionKeys; + } + + @Override + public Map getOptions() { + return options; + } + + @Override + public String getComment() { + return comment != null ? comment : ""; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 09fc0328ef65..ff6013e3fed0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -24,6 +24,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; +import org.apache.paimon.flink.utils.FlinkDescriptorProperties; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.operation.FileStoreCommit; @@ -46,7 +47,6 @@ import org.apache.paimon.view.View; import org.apache.paimon.view.ViewImpl; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.AbstractCatalog; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabase; @@ -96,7 +96,6 @@ import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; -import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.factories.Factory; import org.apache.flink.table.procedures.Procedure; @@ -121,13 +120,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; import static org.apache.flink.table.utils.EncodingUtils.decodeBase64ToBytes; @@ -152,11 +144,18 @@ import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; import static org.apache.paimon.flink.log.LogStoreRegister.registerLogSystem; import static org.apache.paimon.flink.log.LogStoreRegister.unRegisterLogSystem; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeWatermarkSpec; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.nonPhysicalColumnsCount; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.serializeNewWatermarkSpec; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.COMMENT; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; import static org.apache.paimon.flink.utils.TableStatsUtil.createTableColumnStats; import static org.apache.paimon.flink.utils.TableStatsUtil.createTableStats; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -1002,18 +1001,18 @@ private static void validateAlterTable(CatalogBaseTable ct1, CatalogBaseTable ct } // materialized table is not resolved at this time. if (!table1IsMaterialized) { - org.apache.flink.table.api.TableSchema ts1 = ct1.getSchema(); - org.apache.flink.table.api.TableSchema ts2 = ct2.getSchema(); + org.apache.flink.table.api.Schema ts1 = ct1.getUnresolvedSchema(); + org.apache.flink.table.api.Schema ts2 = ct2.getUnresolvedSchema(); boolean pkEquality = false; if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { pkEquality = Objects.equals( - ts1.getPrimaryKey().get().getType(), - ts2.getPrimaryKey().get().getType()) + ts1.getPrimaryKey().get().getConstraintName(), + ts2.getPrimaryKey().get().getConstraintName()) && Objects.equals( - ts1.getPrimaryKey().get().getColumns(), - ts2.getPrimaryKey().get().getColumns()); + ts1.getPrimaryKey().get().getColumnNames(), + ts2.getPrimaryKey().get().getColumnNames()); } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { pkEquality = true; } @@ -1057,7 +1056,8 @@ public final void close() throws CatalogException { private CatalogBaseTable toCatalogTable(Table table) { Map newOptions = new HashMap<>(table.options()); - TableSchema.Builder builder = TableSchema.builder(); + org.apache.flink.table.api.Schema.Builder builder = + org.apache.flink.table.api.Schema.newBuilder(); Map nonPhysicalColumnComments = new HashMap<>(); // add columns @@ -1072,10 +1072,10 @@ private CatalogBaseTable toCatalogTable(Table table) { if (optionalName == null || physicalColumns.contains(optionalName)) { // build physical column from table row field RowType.RowField field = physicalRowFields.get(physicalColumnIndex++); - builder.field(field.getName(), fromLogicalToDataType(field.getType())); + builder.column(field.getName(), fromLogicalToDataType(field.getType())); } else { // build non-physical column from options - builder.add(deserializeNonPhysicalColumn(newOptions, i)); + deserializeNonPhysicalColumn(newOptions, i, builder); if (newOptions.containsKey(compoundKey(SCHEMA, i, COMMENT))) { nonPhysicalColumnComments.put( optionalName, newOptions.get(compoundKey(SCHEMA, i, COMMENT))); @@ -1087,22 +1087,18 @@ private CatalogBaseTable toCatalogTable(Table table) { // extract watermark information if (newOptions.keySet().stream() .anyMatch(key -> key.startsWith(compoundKey(SCHEMA, WATERMARK)))) { - builder.watermark(deserializeWatermarkSpec(newOptions)); + deserializeWatermarkSpec(newOptions, builder); } // add primary keys if (table.primaryKeys().size() > 0) { - builder.primaryKey( - table.primaryKeys().stream().collect(Collectors.joining("_", "PK_", "")), - table.primaryKeys().toArray(new String[0])); + builder.primaryKey(table.primaryKeys()); } - TableSchema schema = builder.build(); + org.apache.flink.table.api.Schema schema = builder.build(); // remove schema from options - DescriptorProperties removeProperties = new DescriptorProperties(false); - removeProperties.putTableSchema(SCHEMA, schema); - removeProperties.asMap().keySet().forEach(newOptions::remove); + FlinkDescriptorProperties.removeSchemaKeys(SCHEMA, schema, newOptions); Options options = Options.fromMap(newOptions); if (TableType.MATERIALIZED_TABLE == options.get(CoreOptions.TYPE)) { @@ -1118,7 +1114,10 @@ private CatalogBaseTable toCatalogTable(Table table) { } private CatalogMaterializedTable buildMaterializedTable( - Table table, Map newOptions, TableSchema schema, Options options) { + Table table, + Map newOptions, + org.apache.flink.table.api.Schema schema, + Options options) { String definitionQuery = options.get(MATERIALIZED_TABLE_DEFINITION_QUERY); IntervalFreshness freshness = IntervalFreshness.of( @@ -1142,7 +1141,7 @@ private CatalogMaterializedTable buildMaterializedTable( // remove materialized table related options allMaterializedTableAttributes().forEach(newOptions::remove); return CatalogMaterializedTable.newBuilder() - .schema(schema.toSchema()) + .schema(schema) .comment(table.comment().orElse("")) .partitionKeys(table.partitionKeys()) .options(newOptions) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java index 37bed2d0480f..75af5917bb49 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java @@ -48,7 +48,6 @@ import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.factories.Factory; import org.apache.flink.table.factories.FunctionDefinitionFactory; -import org.apache.flink.table.factories.TableFactory; import org.apache.flink.table.procedures.Procedure; import java.util.List; @@ -86,11 +85,6 @@ public Optional getFactory() { new FlinkGenericTableFactory(paimon.getFactory().get(), flink.getFactory().get())); } - @Override - public Optional getTableFactory() { - return flink.getTableFactory(); - } - @Override public Optional getFunctionDefinitionFactory() { return flink.getFunctionDefinitionFactory(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java index d5d843d91bb1..f88a808713c2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java @@ -22,7 +22,6 @@ import org.apache.paimon.table.system.AuditLogTable; import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.WatermarkSpec; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.types.utils.TypeConversions; @@ -32,11 +31,11 @@ import java.util.Map; import java.util.Optional; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeWatermarkSpec; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; /** A {@link CatalogTable} to represent system table. */ public class SystemCatalogTable implements CatalogTable { @@ -60,11 +59,8 @@ public Schema getUnresolvedSchema() { Map newOptions = new HashMap<>(table.options()); if (newOptions.keySet().stream() .anyMatch(key -> key.startsWith(compoundKey(SCHEMA, WATERMARK)))) { - WatermarkSpec watermarkSpec = deserializeWatermarkSpec(newOptions); - return builder.watermark( - watermarkSpec.getRowtimeAttribute(), - watermarkSpec.getWatermarkExpr()) - .build(); + deserializeWatermarkSpec(newOptions, builder); + return builder.build(); } } return builder.build(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java index b0f99a6e89e4..0ec348a00178 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java @@ -20,10 +20,10 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableSet; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.WatermarkSpec; +import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.WatermarkSpec; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.utils.LogicalTypeParser; import org.apache.flink.table.types.utils.TypeConversions; @@ -36,41 +36,43 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.METADATA; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.VIRTUAL; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.COMMENT; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.EXPR; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.METADATA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.VIRTUAL; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; /** * Utilities for ser/deserializing non-physical columns and watermark into/from a map of string * properties. */ public class FlinkCatalogPropertiesUtil { + public static final String SCHEMA = "schema"; public static Map serializeNonPhysicalColumns( - Map indexMap, List nonPhysicalColumns) { + Map indexMap, List nonPhysicalColumns) { Map serialized = new HashMap<>(); - for (TableColumn c : nonPhysicalColumns) { + for (Column c : nonPhysicalColumns) { int index = indexMap.get(c.getName()); serialized.put(compoundKey(SCHEMA, index, NAME), c.getName()); serialized.put( compoundKey(SCHEMA, index, DATA_TYPE), - c.getType().getLogicalType().asSerializableString()); - if (c instanceof TableColumn.ComputedColumn) { - TableColumn.ComputedColumn computedColumn = (TableColumn.ComputedColumn) c; - serialized.put(compoundKey(SCHEMA, index, EXPR), computedColumn.getExpression()); + c.getDataType().getLogicalType().asSerializableString()); + if (c instanceof Column.ComputedColumn) { + Column.ComputedColumn computedColumn = (Column.ComputedColumn) c; + serialized.put( + compoundKey(SCHEMA, index, EXPR), + computedColumn.getExpression().asSerializableString()); } else { - TableColumn.MetadataColumn metadataColumn = (TableColumn.MetadataColumn) c; + Column.MetadataColumn metadataColumn = (Column.MetadataColumn) c; serialized.put( compoundKey(SCHEMA, index, METADATA), - metadataColumn.getMetadataAlias().orElse(metadataColumn.getName())); + metadataColumn.getMetadataKey().orElse(metadataColumn.getName())); serialized.put( compoundKey(SCHEMA, index, VIRTUAL), Boolean.toString(metadataColumn.isVirtual())); @@ -127,10 +129,14 @@ public static Map serializeWatermarkSpec(WatermarkSpec watermark watermarkSpec.getRowtimeAttribute()); serializedWatermarkSpec.put( compoundKey(watermarkPrefix, WATERMARK_STRATEGY_EXPR), - watermarkSpec.getWatermarkExpr()); + watermarkSpec.getWatermarkExpression().asSerializableString()); serializedWatermarkSpec.put( compoundKey(watermarkPrefix, WATERMARK_STRATEGY_DATA_TYPE), - watermarkSpec.getWatermarkExprOutputType().getLogicalType().asSerializableString()); + watermarkSpec + .getWatermarkExpression() + .getOutputDataType() + .getLogicalType() + .asSerializableString()); return serializedWatermarkSpec; } @@ -219,7 +225,8 @@ private static boolean isColumnNameKey(String key) { && SCHEMA_COLUMN_NAME_SUFFIX.matcher(key.substring(SCHEMA.length() + 1)).matches(); } - public static TableColumn deserializeNonPhysicalColumn(Map options, int index) { + public static void deserializeNonPhysicalColumn( + Map options, int index, Schema.Builder builder) { String nameKey = compoundKey(SCHEMA, index, NAME); String dataTypeKey = compoundKey(SCHEMA, index, DATA_TYPE); String exprKey = compoundKey(SCHEMA, index, EXPR); @@ -227,45 +234,42 @@ public static TableColumn deserializeNonPhysicalColumn(Map optio String virtualKey = compoundKey(SCHEMA, index, VIRTUAL); String name = options.get(nameKey); - DataType dataType = - TypeConversions.fromLogicalToDataType( - LogicalTypeParser.parse(options.get(dataTypeKey))); - TableColumn column; if (options.containsKey(exprKey)) { - column = TableColumn.computed(name, dataType, options.get(exprKey)); + final String expr = options.get(exprKey); + builder.columnByExpression(name, expr); } else if (options.containsKey(metadataKey)) { String metadataAlias = options.get(metadataKey); boolean isVirtual = Boolean.parseBoolean(options.get(virtualKey)); - column = - metadataAlias.equals(name) - ? TableColumn.metadata(name, dataType, isVirtual) - : TableColumn.metadata(name, dataType, metadataAlias, isVirtual); + DataType dataType = + TypeConversions.fromLogicalToDataType( + LogicalTypeParser.parse( + options.get(dataTypeKey), + Thread.currentThread().getContextClassLoader())); + if (metadataAlias.equals(name)) { + builder.columnByMetadata(name, dataType, isVirtual); + } else { + builder.columnByMetadata(name, dataType, metadataAlias, isVirtual); + } } else { throw new RuntimeException( String.format( "Failed to build non-physical column. Current index is %s, options are %s", index, options)); } - - return column; } - public static WatermarkSpec deserializeWatermarkSpec(Map options) { + public static void deserializeWatermarkSpec( + Map options, Schema.Builder builder) { String watermarkPrefixKey = compoundKey(SCHEMA, WATERMARK); String rowtimeKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_ROWTIME); String exprKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_STRATEGY_EXPR); - String dataTypeKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_STRATEGY_DATA_TYPE); String rowtimeAttribute = options.get(rowtimeKey); String watermarkExpressionString = options.get(exprKey); - DataType watermarkExprOutputType = - TypeConversions.fromLogicalToDataType( - LogicalTypeParser.parse(options.get(dataTypeKey))); - return new WatermarkSpec( - rowtimeAttribute, watermarkExpressionString, watermarkExprOutputType); + builder.watermark(rowtimeAttribute, watermarkExpressionString); } public static String compoundKey(Object... components) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java new file mode 100644 index 000000000000..edc73ca7bf41 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java @@ -0,0 +1,99 @@ +/* + * 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.table.api.Schema; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utility class for having a unified string-based representation of Table API related classes such + * as Schema, TypeInformation, etc. + * + *

Note to implementers: Please try to reuse key names as much as possible. Key-names should be + * hierarchical and lower case. Use "-" instead of dots or camel case. E.g., + * connector.schema.start-from = from-earliest. Try not to use the higher level in a key-name. E.g., + * instead of connector.kafka.kafka-version use connector.kafka.version. + * + *

Properties with key normalization enabled contain only lower-case keys. + */ +public class FlinkDescriptorProperties { + + public static final String NAME = "name"; + + public static final String DATA_TYPE = "data-type"; + + public static final String EXPR = "expr"; + + public static final String METADATA = "metadata"; + + public static final String VIRTUAL = "virtual"; + + public static final String WATERMARK = "watermark"; + + public static final String WATERMARK_ROWTIME = "rowtime"; + + public static final String WATERMARK_STRATEGY = "strategy"; + + public static final String WATERMARK_STRATEGY_EXPR = WATERMARK_STRATEGY + '.' + EXPR; + + public static final String WATERMARK_STRATEGY_DATA_TYPE = WATERMARK_STRATEGY + '.' + DATA_TYPE; + + public static final String PRIMARY_KEY_NAME = "primary-key.name"; + + public static final String PRIMARY_KEY_COLUMNS = "primary-key.columns"; + + public static final String COMMENT = "comment"; + + public static void removeSchemaKeys(String key, Schema schema, Map options) { + checkNotNull(key); + checkNotNull(schema); + + List subKeys = Arrays.asList(NAME, DATA_TYPE, EXPR, METADATA, VIRTUAL); + for (int idx = 0; idx < schema.getColumns().size(); idx++) { + for (String subKey : subKeys) { + options.remove(key + '.' + idx + '.' + subKey); + } + } + + if (!schema.getWatermarkSpecs().isEmpty()) { + subKeys = + Arrays.asList( + WATERMARK_ROWTIME, + WATERMARK_STRATEGY_EXPR, + WATERMARK_STRATEGY_DATA_TYPE); + for (int idx = 0; idx < schema.getWatermarkSpecs().size(); idx++) { + for (String subKey : subKeys) { + options.remove(key + '.' + WATERMARK + '.' + idx + '.' + subKey); + } + } + } + + schema.getPrimaryKey() + .ifPresent( + pk -> { + options.remove(key + '.' + PRIMARY_KEY_NAME); + options.remove(key + '.' + PRIMARY_KEY_COLUMNS); + }); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java index 9268a236b6cb..dcb5be2262d7 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java @@ -21,27 +21,34 @@ import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.WatermarkSpec; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.WatermarkSpec; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.SqlCallExpression; +import org.apache.flink.table.types.DataType; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import static org.apache.flink.table.descriptors.DescriptorProperties.DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.METADATA; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.VIRTUAL; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.EXPR; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.METADATA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.VIRTUAL; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link FlinkCatalogPropertiesUtil}. */ @@ -53,14 +60,23 @@ public void testSerDeNonPhysicalColumns() { indexMap.put("comp", 2); indexMap.put("meta1", 3); indexMap.put("meta2", 5); - List columns = new ArrayList<>(); - columns.add(TableColumn.computed("comp", DataTypes.INT(), "`k` * 2")); - columns.add(TableColumn.metadata("meta1", DataTypes.VARCHAR(10))); - columns.add(TableColumn.metadata("meta2", DataTypes.BIGINT().notNull(), "price", true)); + List columns = new ArrayList<>(); + columns.add(new Schema.UnresolvedComputedColumn("comp", new SqlCallExpression("`k` * 2"))); + columns.add( + new Schema.UnresolvedMetadataColumn("meta1", DataTypes.VARCHAR(10), null, false)); + columns.add( + new Schema.UnresolvedMetadataColumn( + "meta2", DataTypes.BIGINT().notNull(), "price", true, null)); + + List resolvedColumns = new ArrayList<>(); + resolvedColumns.add( + Column.computed("comp", new TestResolvedExpression("`k` * 2", DataTypes.INT()))); + resolvedColumns.add(Column.metadata("meta1", DataTypes.VARCHAR(10), null, false)); + resolvedColumns.add(Column.metadata("meta2", DataTypes.BIGINT().notNull(), "price", true)); // validate serialization Map serialized = - FlinkCatalogPropertiesUtil.serializeNonPhysicalColumns(indexMap, columns); + FlinkCatalogPropertiesUtil.serializeNonPhysicalColumns(indexMap, resolvedColumns); Map expected = new HashMap<>(); expected.put(compoundKey(SCHEMA, 2, NAME), "comp"); @@ -80,12 +96,13 @@ public void testSerDeNonPhysicalColumns() { assertThat(serialized).containsExactlyInAnyOrderEntriesOf(expected); // validate deserialization - List deserialized = new ArrayList<>(); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 2)); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 3)); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 5)); + Schema.Builder builder = Schema.newBuilder(); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 2, builder); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 3, builder); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 5, builder); - assertThat(deserialized).isEqualTo(columns); + assertThat(builder.build().getColumns()) + .containsExactly(columns.toArray(new Schema.UnresolvedColumn[0])); // validate that } @@ -93,10 +110,10 @@ public void testSerDeNonPhysicalColumns() { @Test public void testSerDeWatermarkSpec() { WatermarkSpec watermarkSpec = - new WatermarkSpec( + WatermarkSpec.of( "test_time", - "`test_time` - INTERVAL '0.001' SECOND", - DataTypes.TIMESTAMP(3)); + new TestResolvedExpression( + "`test_time` - INTERVAL '0.001' SECOND", DataTypes.TIMESTAMP(3))); // validate serialization Map serialized = @@ -113,9 +130,13 @@ public void testSerDeWatermarkSpec() { assertThat(serialized).containsExactlyInAnyOrderEntriesOf(expected); // validate serialization - WatermarkSpec deserialized = - FlinkCatalogPropertiesUtil.deserializeWatermarkSpec(serialized); - assertThat(deserialized).isEqualTo(watermarkSpec); + Schema.Builder builder = Schema.newBuilder(); + FlinkCatalogPropertiesUtil.deserializeWatermarkSpec(serialized, builder); + assertThat(builder.build().getWatermarkSpecs()).hasSize(1); + Schema.UnresolvedWatermarkSpec actual = builder.build().getWatermarkSpecs().get(0); + assertThat(actual.getColumnName()).isEqualTo(watermarkSpec.getRowtimeAttribute()); + assertThat(actual.getWatermarkExpression().asSummaryString()) + .isEqualTo(watermarkSpec.getWatermarkExpression().asSummaryString()); } @Test @@ -150,4 +171,44 @@ public void testNonPhysicalColumnsCount() { oldStyleOptions, Arrays.asList("phy1", "phy2"))) .isEqualTo(3); } + + private static class TestResolvedExpression implements ResolvedExpression { + private final String name; + private final DataType outputDataType; + + private TestResolvedExpression(String name, DataType outputDataType) { + this.name = name; + this.outputDataType = outputDataType; + } + + @Override + public DataType getOutputDataType() { + return outputDataType; + } + + @Override + public List getResolvedChildren() { + return Collections.emptyList(); + } + + @Override + public String asSummaryString() { + return new SqlCallExpression(name).asSummaryString(); + } + + @Override + public String asSerializableString() { + return name; + } + + @Override + public List getChildren() { + return Collections.emptyList(); + } + + @Override + public R accept(ExpressionVisitor expressionVisitor) { + return null; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index 27a89510975f..e4286eb18172 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -850,7 +850,7 @@ private static void checkEquals(CatalogBaseTable t1, CatalogBaseTable t2) { assertThat(t2.getComment()).isEqualTo(t1.getComment()); assertThat(t2.getOptions()).isEqualTo(t1.getOptions()); if (t1.getTableKind() == CatalogBaseTable.TableKind.TABLE) { - assertThat(t2.getSchema()).isEqualTo(t1.getSchema()); + assertThat(t2.getUnresolvedSchema()).isEqualTo(t1.getUnresolvedSchema()); assertThat(((CatalogTable) (t2)).getPartitionKeys()) .isEqualTo(((CatalogTable) (t1)).getPartitionKeys()); assertThat(((CatalogTable) (t2)).isPartitioned()) @@ -864,7 +864,12 @@ private static void checkEquals(CatalogBaseTable t1, CatalogBaseTable t2) { t2.getUnresolvedSchema() .resolve(new TestSchemaResolver())) .build()) - .isEqualTo(t1.getSchema().toSchema()); + .isEqualTo( + Schema.newBuilder() + .fromResolvedSchema( + t1.getUnresolvedSchema() + .resolve(new TestSchemaResolver())) + .build()); assertThat(mt2.getPartitionKeys()).isEqualTo(mt1.getPartitionKeys()); assertThat(mt2.isPartitioned()).isEqualTo(mt1.isPartitioned()); // validate definition query From 821894a222bb7854164f51e38293a745d254a966 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:36:42 +0800 Subject: [PATCH 13/17] [flink] Avoid deprecated usage of DiscardingSink --- .../org/apache/flink/api/common/JobInfo.java | 40 ++++++ .../SupportsConcurrentExecutionAttempts.java | 31 +++++ .../api/connector/sink2/InitContext.java | 120 ++++++++++++++++++ .../connector/sink2/WriterInitContext.java | 83 ++++++++++++ .../api/functions/sink/v2/DiscardingSink.java | 59 +++++++++ .../org/apache/flink/api/common/JobInfo.java | 40 ++++++ .../SupportsConcurrentExecutionAttempts.java | 31 +++++ .../api/connector/sink2/InitContext.java | 120 ++++++++++++++++++ .../connector/sink2/WriterInitContext.java | 83 ++++++++++++ .../api/functions/sink/v2/DiscardingSink.java | 59 +++++++++ .../org/apache/flink/api/common/JobInfo.java | 40 ++++++ .../api/connector/sink2/InitContext.java | 120 ++++++++++++++++++ .../connector/sink2/WriterInitContext.java | 83 ++++++++++++ .../api/functions/sink/v2/DiscardingSink.java | 59 +++++++++ .../org/apache/flink/api/common/JobInfo.java | 40 ++++++ .../api/connector/sink2/InitContext.java | 120 ++++++++++++++++++ .../connector/sink2/WriterInitContext.java | 83 ++++++++++++ .../api/functions/sink/v2/DiscardingSink.java | 59 +++++++++ .../sink/cdc/FlinkCdcMultiTableSink.java | 4 +- .../sink/cdc/FlinkCdcMultiTableSinkTest.java | 5 +- .../paimon/flink/action/CloneAction.java | 4 +- .../sink/CombinedTableCompactorSink.java | 4 +- .../apache/paimon/flink/sink/FlinkSink.java | 4 +- 23 files changed, 1280 insertions(+), 11 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/JobInfo.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/SupportsConcurrentExecutionAttempts.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/JobInfo.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/SupportsConcurrentExecutionAttempts.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/JobInfo.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/JobInfo.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/InitContext.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java 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/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/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-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/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/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.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/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/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.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/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/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.18/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.18/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-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index c13928ab164e..e607e1b9e372 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -40,7 +40,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import javax.annotation.Nullable; @@ -137,7 +137,7 @@ public DataStreamSink sinkFrom( createCommittableStateManager())) .setParallelism(input.getParallelism()); configureGlobalCommitter(committed, commitCpuCores, commitHeapMemory); - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } protected OneInputStreamOperator createWriteOperator( diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java index fd23e500d5e5..e50661e0c655 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java @@ -22,11 +22,11 @@ import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.options.Options; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; -import org.apache.flink.streaming.api.transformations.LegacySinkTransformation; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.junit.jupiter.api.Test; @@ -65,8 +65,7 @@ public void cancel() {} DataStreamSink dataStreamSink = sink.sinkFrom(input); // check the transformation graph - LegacySinkTransformation end = - (LegacySinkTransformation) dataStreamSink.getTransformation(); + Transformation end = dataStreamSink.getTransformation(); assertThat(end.getName()).isEqualTo("end"); OneInputTransformation committer = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java index 2f90147eeb2a..bac030dd0496 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java @@ -32,7 +32,7 @@ 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.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import java.util.HashMap; import java.util.Map; @@ -141,7 +141,7 @@ copyFiles, new SnapshotHintChannelComputer(), parallelism) new SnapshotHintOperator(targetCatalogConfig)) .setParallelism(parallelism); - snapshotHintOperator.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + snapshotHintOperator.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java index fc84dd671469..a04914b71e24 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java @@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.data.RowData; @@ -164,7 +164,7 @@ protected DataStreamSink doCommit( if (!options.get(SINK_COMMITTER_OPERATOR_CHAINING)) { committed = committed.startNewChain(); } - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } // TODO:refactor FlinkSink to adopt this sink diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index 214e2858ae53..9543182643a9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -44,7 +44,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -318,7 +318,7 @@ protected DataStreamSink doCommit(DataStream written, String com } configureGlobalCommitter( committed, options.get(SINK_COMMITTER_CPU), options.get(SINK_COMMITTER_MEMORY)); - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } public static void configureGlobalCommitter( From 9cea55374d30693eee0e96a4e017ae5e88490c55 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:37:02 +0800 Subject: [PATCH 14/17] [flink] Replace legacy SinkFunction with v2 Sink --- .../flink/service/QueryAddressRegister.java | 84 +++++++++++-------- .../paimon/flink/service/QueryService.java | 2 +- 2 files changed, 48 insertions(+), 38 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java index 524f2e5f01c1..00d527506cfe 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java @@ -23,10 +23,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.WriterInitContext; import java.net.InetSocketAddress; import java.util.TreeMap; @@ -34,57 +33,68 @@ import static org.apache.paimon.service.ServiceManager.PRIMARY_KEY_LOOKUP; /** Operator for address server to register addresses to {@link ServiceManager}. */ -public class QueryAddressRegister extends RichSinkFunction { - +public class QueryAddressRegister implements Sink { private final ServiceManager serviceManager; - private transient int numberExecutors; - private transient TreeMap executors; - public QueryAddressRegister(Table table) { this.serviceManager = ((FileStoreTable) table).store().newServiceManager(); } /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + public SinkWriter createWriter(InitContext context) { + return new QueryAddressRegisterSinkWriter(serviceManager); } /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. */ - public void open(Configuration parameters) throws Exception { - this.executors = new TreeMap<>(); + public SinkWriter createWriter(WriterInitContext context) { + return new QueryAddressRegisterSinkWriter(serviceManager); } - @Override - public void invoke(InternalRow row, SinkFunction.Context context) { - int numberExecutors = row.getInt(0); - if (this.numberExecutors != 0 && this.numberExecutors != numberExecutors) { - throw new IllegalArgumentException( - String.format( - "Number Executors can not be changed! Old %s , New %s .", - this.numberExecutors, numberExecutors)); - } - this.numberExecutors = numberExecutors; + private static class QueryAddressRegisterSinkWriter implements SinkWriter { + private final ServiceManager serviceManager; - int executorId = row.getInt(1); - String hostname = row.getString(2).toString(); - int port = row.getInt(3); + private final TreeMap executors; - executors.put(executorId, new InetSocketAddress(hostname, port)); + private int numberExecutors; - if (executors.size() == numberExecutors) { - serviceManager.resetService( - PRIMARY_KEY_LOOKUP, executors.values().toArray(new InetSocketAddress[0])); + private QueryAddressRegisterSinkWriter(ServiceManager serviceManager) { + this.serviceManager = serviceManager; + this.executors = new TreeMap<>(); } - } - @Override - public void close() throws Exception { - super.close(); - serviceManager.deleteService(PRIMARY_KEY_LOOKUP); + @Override + public void write(InternalRow row, Context context) { + int numberExecutors = row.getInt(0); + if (this.numberExecutors != 0 && this.numberExecutors != numberExecutors) { + throw new IllegalArgumentException( + String.format( + "Number Executors can not be changed! Old %s , New %s .", + this.numberExecutors, numberExecutors)); + } + this.numberExecutors = numberExecutors; + + int executorId = row.getInt(1); + String hostname = row.getString(2).toString(); + int port = row.getInt(3); + + executors.put(executorId, new InetSocketAddress(hostname, port)); + + if (executors.size() == numberExecutors) { + serviceManager.resetService( + PRIMARY_KEY_LOOKUP, executors.values().toArray(new InetSocketAddress[0])); + } + } + + @Override + public void flush(boolean endOfInput) {} + + @Override + public void close() { + serviceManager.deleteService(PRIMARY_KEY_LOOKUP); + } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java index bd433fe0f00d..752d54cff5a0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java @@ -62,7 +62,7 @@ public static void build(StreamExecutionEnvironment env, Table table, int parall InternalTypeInfo.fromRowType(QueryExecutorOperator.outputType()), executorOperator) .setParallelism(parallelism) - .addSink(new QueryAddressRegister(table)) + .sinkTo(new QueryAddressRegister(table)) .setParallelism(1); sink.getTransformation().setMaxParallelism(1); From 2f8c6675f7144282ff44c109144730f3026ceec7 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 18 Nov 2024 11:37:42 +0800 Subject: [PATCH 15/17] [flink] Replace legacy SourceFunction with v2 Source --- .../flink/source/operator/MonitorSource.java} | 8 +- .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ paimon-flink/paimon-flink-1.20/pom.xml | 14 + .../flink/source/operator/MonitorSource.java | 277 ++++++++++++++++++ .../action/cdc/SynchronizationActionBase.java | 62 ++-- .../sink/cdc/FlinkCdcMultiTableSinkTest.java | 10 +- .../cdc/FlinkCdcSyncDatabaseSinkITCase.java | 6 +- .../sink/cdc/FlinkCdcSyncTableSinkITCase.java | 6 +- .../paimon/flink/sink/cdc/TestCdcSource.java | 123 ++++++++ .../flink/sink/cdc/TestCdcSourceFunction.java | 107 ------- .../compact/MultiAwareBucketTableScan.java | 12 +- .../flink/compact/MultiTableScanBase.java | 14 +- .../compact/MultiUnawareBucketTableScan.java | 12 +- .../UnawareBucketCompactionTopoBuilder.java | 2 +- .../flink/service/QueryFileMonitor.java | 99 +++---- .../source/AbstractNonCoordinatedSource.java | 50 ++++ .../AbstractNonCoordinatedSourceReader.java | 51 ++++ .../source/BucketUnawareCompactSource.java | 122 ++++---- .../CombinedTableCompactorSourceBuilder.java | 16 +- .../flink/source/FlinkSourceBuilder.java | 4 +- .../paimon/flink/source/NoOpEnumState.java | 22 ++ .../flink/source/NoOpEnumStateSerializer.java | 41 +++ .../paimon/flink/source/NoOpEnumerator.java | 54 ++++ .../flink/source/SimpleSourceSplit.java | 39 +++ .../source/SimpleSourceSplitSerializer.java | 45 +++ ...ion.java => CombinedAwareBatchSource.java} | 95 +++--- ...java => CombinedAwareStreamingSource.java} | 98 +++---- ...tion.java => CombinedCompactorSource.java} | 47 +-- ...n.java => CombinedUnawareBatchSource.java} | 103 ++++--- ...va => CombinedUnawareStreamingSource.java} | 102 +++---- .../flink/source/operator/MonitorSource.java | 248 ++++++++++++++++ .../operator/MultiTablesReadOperator.java | 5 +- .../MultiUnawareTablesReadOperator.java | 2 +- .../flink/source/operator/ReadOperator.java | 4 +- .../apache/paimon/flink/FileStoreITCase.java | 18 +- .../apache/paimon/flink/FiniteTestSource.java | 176 ++++++----- .../paimon/flink/SerializableRowData.java | 4 +- .../UnawareBucketAppendOnlyTableITCase.java | 90 +++--- .../source/operator/OperatorSourceTest.java | 113 ++++--- .../operator/TestingSourceOperator.java | 175 +++++++++++ 43 files changed, 2829 insertions(+), 755 deletions(-) rename paimon-flink/{paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java => paimon-flink-1.15/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java} (98%) create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-1.19/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-1.20/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java delete mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedAwareBatchSourceFunction.java => CombinedAwareBatchSource.java} (66%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedAwareStreamingSourceFunction.java => CombinedAwareStreamingSource.java} (63%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedCompactorSourceFunction.java => CombinedCompactorSource.java} (63%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedUnawareBatchSourceFunction.java => CombinedUnawareBatchSource.java} (71%) rename paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/{CombinedUnawareStreamingSourceFunction.java => CombinedUnawareStreamingSource.java} (57%) create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java 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.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: + * + *

    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. 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: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. 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/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: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. 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: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. 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.18/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.18/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.18/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: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. 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: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. 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.19/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.19/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.19/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: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. 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: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. 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.20/pom.xml b/paimon-flink/paimon-flink-1.20/pom.xml index 7cf1d8e98df7..f15792d2bea9 100644 --- a/paimon-flink/paimon-flink-1.20/pom.xml +++ b/paimon-flink/paimon-flink-1.20/pom.xml @@ -55,6 +55,20 @@ under the License.
+ + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + org.apache.flink + flink-table-common + ${flink.version} + provided + diff --git a/paimon-flink/paimon-flink-1.20/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-1.20/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.20/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: + * + *
    + *
  1. Monitoring snapshots of the Paimon table. + *
  2. Creating the {@link Split splits} corresponding to the incremental files + *
  3. 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: + * + *

    + *
  1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
  2. 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-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index f103396389e5..a7c770347410 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -40,7 +40,6 @@ import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import java.time.Duration; import java.util.HashMap; @@ -131,7 +130,7 @@ public void build() throws Exception { protected void beforeBuildingSourceSink() throws Exception {} - protected Object buildSource() { + protected Source buildSource() { return syncJobHandler.provideSource(); } @@ -147,41 +146,32 @@ protected void validateRuntimeExecutionMode() { "It's only support STREAMING mode for flink-cdc sync table action."); } - private DataStreamSource buildDataStreamSource(Object source) { - if (source instanceof Source) { - boolean isAutomaticWatermarkCreationEnabled = - tableConfig.containsKey(CoreOptions.TAG_AUTOMATIC_CREATION.key()) - && Objects.equals( - tableConfig.get(CoreOptions.TAG_AUTOMATIC_CREATION.key()), - WATERMARK.toString()); - - Options options = Options.fromMap(tableConfig); - Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); - String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); - WatermarkStrategy watermarkStrategy = - isAutomaticWatermarkCreationEnabled - ? watermarkAlignGroup != null - ? new CdcWatermarkStrategy(createCdcTimestampExtractor()) - .withWatermarkAlignment( - watermarkAlignGroup, - options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), - options.get( - SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)) - : new CdcWatermarkStrategy(createCdcTimestampExtractor()) - : WatermarkStrategy.noWatermarks(); - if (idleTimeout != null) { - watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); - } - return env.fromSource( - (Source) source, - watermarkStrategy, - syncJobHandler.provideSourceName()); + private DataStreamSource buildDataStreamSource( + Source source) { + boolean isAutomaticWatermarkCreationEnabled = + tableConfig.containsKey(CoreOptions.TAG_AUTOMATIC_CREATION.key()) + && Objects.equals( + tableConfig.get(CoreOptions.TAG_AUTOMATIC_CREATION.key()), + WATERMARK.toString()); + + Options options = Options.fromMap(tableConfig); + Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); + String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); + WatermarkStrategy watermarkStrategy = + isAutomaticWatermarkCreationEnabled + ? watermarkAlignGroup != null + ? new CdcWatermarkStrategy(createCdcTimestampExtractor()) + .withWatermarkAlignment( + watermarkAlignGroup, + options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), + options.get( + SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)) + : new CdcWatermarkStrategy(createCdcTimestampExtractor()) + : WatermarkStrategy.noWatermarks(); + if (idleTimeout != null) { + watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); } - if (source instanceof SourceFunction) { - return env.addSource( - (SourceFunction) source, syncJobHandler.provideSourceName()); - } - throw new UnsupportedOperationException("Unrecognized source type"); + return env.fromSource(source, watermarkStrategy, syncJobHandler.provideSourceName()); } protected abstract FlatMapFunction recordParse(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java index e50661e0c655..3f58d0ca0fa9 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java @@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.junit.jupiter.api.Test; @@ -45,14 +44,7 @@ public void testTransformationParallelism() { env.setParallelism(8); int inputParallelism = ThreadLocalRandom.current().nextInt(8) + 1; DataStreamSource input = - env.addSource( - new ParallelSourceFunction() { - @Override - public void run(SourceContext ctx) {} - - @Override - public void cancel() {} - }) + env.fromData(CdcMultiplexRecord.class, new CdcMultiplexRecord("", "", null)) .setParallelism(inputParallelism); FlinkCdcMultiTableSink sink = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java index a7c6b2cb6323..28b137a93ed9 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java @@ -42,6 +42,7 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Test; @@ -154,8 +155,9 @@ private void innerTestRandomCdcEvents(Supplier bucket, boolean unawareB .allowRestart(enableFailure) .build(); - TestCdcSourceFunction sourceFunction = new TestCdcSourceFunction(events); - DataStreamSource source = env.addSource(sourceFunction); + TestCdcSource testCdcSource = new TestCdcSource(events); + DataStreamSource source = + env.fromSource(testCdcSource, WatermarkStrategy.noWatermarks(), "TestCdcSource"); source.setParallelism(2); Options catalogOptions = new Options(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java index 081bd7d073d7..8b19391f3eda 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java @@ -43,6 +43,7 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Disabled; @@ -151,8 +152,9 @@ private void innerTestRandomCdcEvents( .allowRestart(enableFailure) .build(); - TestCdcSourceFunction sourceFunction = new TestCdcSourceFunction(testTable.events()); - DataStreamSource source = env.addSource(sourceFunction); + TestCdcSource testCdcSource = new TestCdcSource(testTable.events()); + DataStreamSource source = + env.fromSource(testCdcSource, WatermarkStrategy.noWatermarks(), "TestCdcSource"); source.setParallelism(2); Options catalogOptions = new Options(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java new file mode 100644 index 000000000000..841a9b8515a8 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java @@ -0,0 +1,123 @@ +/* + * 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.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; + +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Testing parallel {@link org.apache.flink.api.connector.source.Source} to produce {@link + * TestCdcEvent}. {@link TestCdcEvent}s with the same key will be produced by the same parallelism. + */ +public class TestCdcSource extends AbstractNonCoordinatedSource { + + private static final long serialVersionUID = 1L; + private final LinkedList events; + + public TestCdcSource(Collection events) { + this.events = new LinkedList<>(events); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext context) { + return new Reader( + context.getIndexOfSubtask(), + context.currentParallelism(), + new LinkedList<>(events)); + } + + private static class Reader extends AbstractNonCoordinatedSourceReader { + private final int subtaskId; + private final int totalSubtasks; + + private final LinkedList events; + + private final int numRecordsPerCheckpoint; + private final AtomicInteger recordsThisCheckpoint; + + private Reader(int subtaskId, int totalSubtasks, LinkedList events) { + this.subtaskId = subtaskId; + this.totalSubtasks = totalSubtasks; + this.events = events; + numRecordsPerCheckpoint = + events.size() / ThreadLocalRandom.current().nextInt(10, 20) + 1; + recordsThisCheckpoint = new AtomicInteger(0); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + if (events.isEmpty()) { + return InputStatus.END_OF_INPUT; + } + + if (recordsThisCheckpoint.get() >= numRecordsPerCheckpoint) { + Thread.sleep(10); + return InputStatus.MORE_AVAILABLE; + } + + TestCdcEvent event = events.poll(); + if (event.records() != null) { + if (Math.abs(event.hashCode()) % totalSubtasks != subtaskId) { + return InputStatus.MORE_AVAILABLE; + } + } + readerOutput.collect(event); + recordsThisCheckpoint.incrementAndGet(); + return InputStatus.MORE_AVAILABLE; + } + + @Override + public List snapshotState(long l) { + recordsThisCheckpoint.set(0); + return Collections.singletonList( + new SimpleSourceSplit(Integer.toString(events.size()))); + } + + @Override + public void addSplits(List list) { + int count = + list.stream() + .map(x -> Integer.parseInt(x.splitId())) + .reduce(Integer::sum) + .orElse(0); + while (events.size() > count) { + events.poll(); + } + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java deleted file mode 100644 index 4e03256a5253..000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -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.functions.source.RichParallelSourceFunction; - -import java.util.Collection; -import java.util.LinkedList; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Testing {@link RichParallelSourceFunction} to produce {@link TestCdcEvent}. {@link TestCdcEvent}s - * with the same key will be produced by the same parallelism. - */ -public class TestCdcSourceFunction extends RichParallelSourceFunction - implements CheckpointedFunction { - - private static final long serialVersionUID = 1L; - - private final LinkedList events; - - private volatile boolean isRunning = true; - private transient int numRecordsPerCheckpoint; - private transient AtomicInteger recordsThisCheckpoint; - private transient ListState remainingEventsCount; - - public TestCdcSourceFunction(Collection events) { - this.events = new LinkedList<>(events); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - numRecordsPerCheckpoint = events.size() / ThreadLocalRandom.current().nextInt(10, 20) + 1; - recordsThisCheckpoint = new AtomicInteger(0); - - remainingEventsCount = - context.getOperatorStateStore() - .getListState(new ListStateDescriptor<>("count", Integer.class)); - - if (context.isRestored()) { - int count = 0; - for (int c : remainingEventsCount.get()) { - count += c; - } - while (events.size() > count) { - events.poll(); - } - } - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - recordsThisCheckpoint.set(0); - remainingEventsCount.clear(); - remainingEventsCount.add(events.size()); - } - - @Override - public void run(SourceContext ctx) throws Exception { - while (isRunning && !events.isEmpty()) { - if (recordsThisCheckpoint.get() >= numRecordsPerCheckpoint) { - Thread.sleep(10); - continue; - } - - synchronized (ctx.getCheckpointLock()) { - TestCdcEvent event = events.poll(); - if (event.records() != null) { - int subtaskId = getRuntimeContext().getIndexOfThisSubtask(); - int totalSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); - if (Math.abs(event.hashCode()) % totalSubtasks != subtaskId) { - continue; - } - } - ctx.collect(event); - recordsThisCheckpoint.incrementAndGet(); - } - } - } - - @Override - public void cancel() { - isRunning = false; - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java index 747995d20d67..88730132ef68 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java @@ -32,7 +32,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -52,15 +51,8 @@ public MultiAwareBucketTableScan( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { - super( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + boolean isStreaming) { + super(catalogLoader, includingPattern, excludingPattern, databasePattern, isStreaming); tablesMap = new HashMap<>(); scansMap = new HashMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java index bd4ffe83a4ca..f5940740b691 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java @@ -26,12 +26,11 @@ import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.table.source.Split; -import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.api.connector.source.ReaderOutput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import static org.apache.paimon.flink.utils.MultiTablesCompactorUtil.shouldCompactTable; @@ -57,7 +56,6 @@ public abstract class MultiTableScanBase implements AutoCloseable { protected transient Catalog catalog; - protected AtomicBoolean isRunning; protected boolean isStreaming; public MultiTableScanBase( @@ -65,14 +63,12 @@ public MultiTableScanBase( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { + boolean isStreaming) { catalog = catalogLoader.load(); this.includingPattern = includingPattern; this.excludingPattern = excludingPattern; this.databasePattern = databasePattern; - this.isRunning = isRunning; this.isStreaming = isStreaming; } @@ -104,13 +100,9 @@ protected void updateTableMap() } } - public ScanResult scanTable(SourceFunction.SourceContext ctx) + public ScanResult scanTable(ReaderOutput ctx) throws Catalog.TableNotExistException, Catalog.DatabaseNotExistException { try { - if (!isRunning.get()) { - return ScanResult.FINISHED; - } - updateTableMap(); List tasks = doScan(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java index 56bf971240e7..da86b93af512 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java @@ -29,7 +29,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; /** @@ -46,15 +45,8 @@ public MultiUnawareBucketTableScan( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { - super( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + boolean isStreaming) { + super(catalogLoader, includingPattern, excludingPattern, databasePattern, isStreaming); tablesMap = new HashMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java index 8c6ed4c9f59e..a572354e8984 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java @@ -126,7 +126,7 @@ private DataStreamSource buildSource() { new BucketUnawareCompactSource( table, isContinuous, scanInterval, partitionPredicate); - return BucketUnawareCompactSource.buildSource(env, source, isContinuous, tableIdentifier); + return BucketUnawareCompactSource.buildSource(env, source, tableIdentifier); } private void sinkFromSource(DataStreamSource input) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java index 02f8a654112e..f0c17f85eceb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java @@ -21,6 +21,9 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.InternalTypeInfo; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; @@ -31,11 +34,14 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.table.system.FileMonitorTable; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import java.util.ArrayList; import java.util.List; @@ -50,19 +56,13 @@ *
  • Assigning them to downstream tasks for further processing. * */ -public class QueryFileMonitor extends RichSourceFunction { +public class QueryFileMonitor extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; private final Table table; private final long monitorInterval; - private transient SourceContext ctx; - private transient StreamTableScan scan; - private transient TableRead read; - - private volatile boolean isRunning = true; - public QueryFileMonitor(Table table) { this.table = table; this.monitorInterval = @@ -71,65 +71,54 @@ public QueryFileMonitor(Table table) { .toMillis(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); - ReadBuilder readBuilder = monitorTable.newReadBuilder(); - this.scan = readBuilder.newStreamScan(); - this.read = readBuilder.newRead(); + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - @Override - public void run(SourceContext ctx) throws Exception { - this.ctx = ctx; - while (isRunning) { - boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - isEmpty = doScan(); - } + private class Reader extends AbstractNonCoordinatedSourceReader { + private transient StreamTableScan scan; + private transient TableRead read; + + @Override + public void start() { + FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); + ReadBuilder readBuilder = monitorTable.newReadBuilder(); + this.scan = readBuilder.newStreamScan(); + this.read = readBuilder.newRead(); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + boolean isEmpty = doScan(readerOutput); if (isEmpty) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; } - } - private boolean doScan() throws Exception { - List records = new ArrayList<>(); - read.createReader(scan.plan()).forEachRemaining(records::add); - records.forEach(ctx::collect); - return records.isEmpty(); - } - - @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; + private boolean doScan(ReaderOutput readerOutput) throws Exception { + List records = new ArrayList<>(); + read.createReader(scan.plan()).forEachRemaining(records::add); + records.forEach(readerOutput::collect); + return records.isEmpty(); } } public static DataStream build(StreamExecutionEnvironment env, Table table) { - return env.addSource( - new QueryFileMonitor(table), - "FileMonitor-" + table.name(), - InternalTypeInfo.fromRowType(FileMonitorTable.getRowType())); + return env.fromSource( + new QueryFileMonitor(table), + WatermarkStrategy.noWatermarks(), + "FileMonitor-" + table.name(), + InternalTypeInfo.fromRowType(FileMonitorTable.getRowType())) + .setParallelism(1); } public static ChannelComputer createChannelComputer() { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java new file mode 100644 index 000000000000..a9a389e837a2 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java @@ -0,0 +1,50 @@ +/* + * 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; + +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +/** {@link Source} that does not require coordination between JobManager and TaskManagers. */ +public abstract class AbstractNonCoordinatedSource + implements Source { + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new NoOpEnumerator<>(); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, NoOpEnumState checkpoint) { + return new NoOpEnumerator<>(); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new SimpleSourceSplitSerializer(); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new NoOpEnumStateSerializer(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java new file mode 100644 index 000000000000..18c278868ffa --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java @@ -0,0 +1,51 @@ +/* + * 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; + +import org.apache.flink.api.connector.source.SourceReader; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** Abstract {@link SourceReader} for {@link AbstractNonCoordinatedSource}. */ +public abstract class AbstractNonCoordinatedSourceReader + implements SourceReader { + @Override + public void start() {} + + @Override + public List snapshotState(long l) { + return Collections.emptyList(); + } + + @Override + public CompletableFuture isAvailable() { + return CompletableFuture.completedFuture(null); + } + + @Override + public void addSplits(List list) {} + + @Override + public void notifyNoMoreSplits() {} + + @Override + public void close() throws Exception {} +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java index 79ee827fe6e4..e588fef0fb79 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java @@ -21,19 +21,19 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; import org.apache.paimon.flink.sink.CompactionTaskTypeInfo; -import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.EndOfScanException; -import org.apache.paimon.utils.Preconditions; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,15 +42,16 @@ import java.util.List; /** - * Source Function for unaware-bucket Compaction. + * Source for unaware-bucket Compaction. * - *

    Note: The function is the source function of unaware-bucket compactor coordinator. It will - * read the latest snapshot continuously by compactionCoordinator, and generate new compaction - * tasks. The source function is used in unaware-bucket compaction job (both stand-alone and - * write-combined). Besides, we don't need to save state in this function, it will invoke a full - * scan when starting up, and scan continuously for the following snapshot. + *

    Note: The function is the source of unaware-bucket compactor coordinator. It will read the + * latest snapshot continuously by compactionCoordinator, and generate new compaction tasks. The + * source is used in unaware-bucket compaction job (both stand-alone and write-combined). Besides, + * we don't need to save state in this source, it will invoke a full scan when starting up, and scan + * continuously for the following snapshot. */ -public class BucketUnawareCompactSource extends RichSourceFunction { +public class BucketUnawareCompactSource + extends AbstractNonCoordinatedSource { private static final Logger LOG = LoggerFactory.getLogger(BucketUnawareCompactSource.class); private static final String COMPACTION_COORDINATOR_NAME = "Compaction Coordinator"; @@ -59,9 +60,6 @@ public class BucketUnawareCompactSource extends RichSourceFunction ctx; - private volatile boolean isRunning = true; public BucketUnawareCompactSource( FileStoreTable table, @@ -74,76 +72,66 @@ public BucketUnawareCompactSource( this.filter = filter; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public Boundedness getBoundedness() { + return streaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - compactionCoordinator = - new UnawareAppendTableCompactionCoordinator(table, streaming, filter); + @Override + public SourceReader createReader( + SourceReaderContext readerContext) throws Exception { Preconditions.checkArgument( - RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) == 1, + readerContext.currentParallelism() == 1, "Compaction Operator parallelism in paimon MUST be one."); + return new BucketUnawareCompactSourceReader(table, streaming, filter, scanInterval); } - @Override - public void run(SourceContext sourceContext) throws Exception { - this.ctx = sourceContext; - while (isRunning) { - boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - try { - // do scan and plan action, emit append-only compaction tasks. - List tasks = compactionCoordinator.run(); - isEmpty = tasks.isEmpty(); - tasks.forEach(ctx::collect); - } catch (EndOfScanException esf) { - LOG.info("Catching EndOfStreamException, the stream is finished."); - return; - } - } + /** BucketUnawareCompactSourceReader. */ + public static class BucketUnawareCompactSourceReader + extends AbstractNonCoordinatedSourceReader { + private final UnawareAppendTableCompactionCoordinator compactionCoordinator; + private final long scanInterval; + private long lastFetchTimeMillis = 0L; - if (isEmpty) { - Thread.sleep(scanInterval); - } + public BucketUnawareCompactSourceReader( + FileStoreTable table, boolean streaming, Predicate filter, long scanInterval) { + this.scanInterval = scanInterval; + compactionCoordinator = + new UnawareAppendTableCompactionCoordinator(table, streaming, filter); } - } - @Override - public void cancel() { - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; + @Override + public InputStatus pollNext(ReaderOutput readerOutput) + throws Exception { + long sleepTimeMillis = scanInterval - System.currentTimeMillis() + lastFetchTimeMillis; + if (sleepTimeMillis > 0) { + Thread.sleep(sleepTimeMillis); + } + + try { + // do scan and plan action, emit append-only compaction tasks. + List tasks = compactionCoordinator.run(); + tasks.forEach(readerOutput::collect); + return InputStatus.MORE_AVAILABLE; + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return InputStatus.END_OF_INPUT; + } finally { + lastFetchTimeMillis = System.currentTimeMillis(); } - } else { - isRunning = false; } } public static DataStreamSource buildSource( StreamExecutionEnvironment env, BucketUnawareCompactSource source, - boolean streaming, String tableIdentifier) { - final StreamSource sourceOperator = - new StreamSource<>(source); return (DataStreamSource) - new DataStreamSource<>( - env, - new CompactionTaskTypeInfo(), - sourceOperator, - false, + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), COMPACTION_COORDINATOR_NAME + " : " + tableIdentifier, - streaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED) + new CompactionTaskTypeInfo()) .setParallelism(1) .setMaxParallelism(1); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java index e5cbbe845ceb..415eddb037df 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java @@ -21,10 +21,10 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.LogicalTypeConversion; -import org.apache.paimon.flink.source.operator.CombinedAwareBatchSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedUnawareBatchSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedUnawareStreamingSourceFunction; +import org.apache.paimon.flink.source.operator.CombinedAwareBatchSource; +import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSource; +import org.apache.paimon.flink.source.operator.CombinedUnawareBatchSource; +import org.apache.paimon.flink.source.operator.CombinedUnawareStreamingSource; import org.apache.paimon.table.system.CompactBucketsTable; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Preconditions; @@ -87,7 +87,7 @@ public DataStream buildAwareBucketTableSource() { Preconditions.checkArgument(env != null, "StreamExecutionEnvironment should not be null."); RowType produceType = CompactBucketsTable.getRowType(); if (isContinuous) { - return CombinedAwareStreamingSourceFunction.buildSource( + return CombinedAwareStreamingSource.buildSource( env, "Combine-MultiBucketTables--StreamingCompactorSource", InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType)), @@ -97,7 +97,7 @@ public DataStream buildAwareBucketTableSource() { databasePattern, monitorInterval); } else { - return CombinedAwareBatchSourceFunction.buildSource( + return CombinedAwareBatchSource.buildSource( env, "Combine-MultiBucketTables-BatchCompactorSource", InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType)), @@ -112,7 +112,7 @@ public DataStream buildAwareBucketTableSource() { public DataStream buildForUnawareBucketsTableSource() { Preconditions.checkArgument(env != null, "StreamExecutionEnvironment should not be null."); if (isContinuous) { - return CombinedUnawareStreamingSourceFunction.buildSource( + return CombinedUnawareStreamingSource.buildSource( env, "Combined-UnawareBucketTables-StreamingCompactorSource", catalogLoader, @@ -121,7 +121,7 @@ public DataStream buildForUnawareBucketsT databasePattern, monitorInterval); } else { - return CombinedUnawareBatchSourceFunction.buildSource( + return CombinedUnawareBatchSource.buildSource( env, "Combined-UnawareBucketTables-BatchCompactorSource", catalogLoader, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java index 6933be18def6..895927f51e93 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java @@ -26,7 +26,7 @@ import org.apache.paimon.flink.log.LogSourceProvider; import org.apache.paimon.flink.sink.FlinkSink; import org.apache.paimon.flink.source.align.AlignedContinuousFileStoreSource; -import org.apache.paimon.flink.source.operator.MonitorFunction; +import org.apache.paimon.flink.source.operator.MonitorSource; import org.apache.paimon.flink.utils.TableScanUtils; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; @@ -307,7 +307,7 @@ private DataStream buildContinuousStreamOperator() { "Cannot limit streaming source, please use batch execution mode."); } dataStream = - MonitorFunction.buildSource( + MonitorSource.buildSource( env, sourceName, produceTypeInfo(), diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java new file mode 100644 index 000000000000..f07317c155aa --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java @@ -0,0 +1,22 @@ +/* + * 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; + +/** The enumerator state class for {@link NoOpEnumerator}. */ +public class NoOpEnumState {} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java new file mode 100644 index 000000000000..89c0ad6ac1f1 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java @@ -0,0 +1,41 @@ +/* + * 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; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; + +/** {@link SimpleVersionedSerializer} for {@link NoOpEnumState}. */ +public class NoOpEnumStateSerializer implements SimpleVersionedSerializer { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(NoOpEnumState obj) throws IOException { + return new byte[0]; + } + + @Override + public NoOpEnumState deserialize(int version, byte[] serialized) throws IOException { + return new NoOpEnumState(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java new file mode 100644 index 000000000000..f29c6d6db76d --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java @@ -0,0 +1,54 @@ +/* + * 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; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; + +/** + * A {@link SplitEnumerator} that provides no functionality. It is basically used for sources that + * does not require a coordinator. + */ +public class NoOpEnumerator + implements SplitEnumerator { + @Override + public void start() {} + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {} + + @Override + public void addSplitsBack(List splits, int subtaskId) {} + + @Override + public void addReader(int subtaskId) {} + + @Override + public NoOpEnumState snapshotState(long checkpointId) throws Exception { + return new NoOpEnumState(); + } + + @Override + public void close() throws IOException {} +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java new file mode 100644 index 000000000000..92d5e4a436d5 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.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.paimon.flink.source; + +import org.apache.flink.api.connector.source.SourceSplit; + +/** A {@link SourceSplit} that provides basic information through splitId. */ +public class SimpleSourceSplit implements SourceSplit { + private final String splitId; + + public SimpleSourceSplit() { + this(""); + } + + public SimpleSourceSplit(String splitId) { + this.splitId = splitId; + } + + @Override + public String splitId() { + return splitId; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java new file mode 100644 index 000000000000..234c8fb3f82e --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java @@ -0,0 +1,45 @@ +/* + * 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; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; + +/** {@link SimpleVersionedSerializer} for {@link SimpleSourceSplit}. */ +public class SimpleSourceSplitSerializer implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(SimpleSourceSplit obj) throws IOException { + String splitId = obj.splitId(); + return splitId == null ? new byte[0] : splitId.getBytes(); + } + + @Override + public SimpleSourceSplit deserialize(int version, byte[] serialized) throws IOException { + return serialized.length == 0 + ? new SimpleSourceSplit() + : new SimpleSourceSplit(new String(serialized)); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java similarity index 66% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java index 2157be51aee4..c3a1258bb176 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java @@ -21,21 +21,23 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.table.data.RowData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,15 +49,11 @@ import static org.apache.paimon.flink.compact.MultiTableScanBase.ScanResult.IS_EMPTY; /** It is responsible for monitoring compactor source of aware bucket table in batch mode. */ -public class CombinedAwareBatchSourceFunction - extends CombinedCompactorSourceFunction> { +public class CombinedAwareBatchSource extends CombinedCompactorSource> { - private static final Logger LOGGER = - LoggerFactory.getLogger(CombinedAwareBatchSourceFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedAwareBatchSource.class); - private MultiTableScanBase> tableScan; - - public CombinedAwareBatchSourceFunction( + public CombinedAwareBatchSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -63,34 +61,33 @@ public CombinedAwareBatchSourceFunction( super(catalogLoader, includingPattern, excludingPattern, databasePattern, false); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public SourceReader, SimpleSourceSplit> createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiAwareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader extends AbstractNonCoordinatedSourceReader> { + private MultiTableScanBase> tableScan; - @Override - void scanTable() throws Exception { - if (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public void start() { + super.start(); + tableScan = + new MultiAwareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext(ReaderOutput> readerOutput) + throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { // Currently, in the combined mode, there are two scan tasks for the table of two @@ -99,6 +96,15 @@ void scanTable() throws Exception { // should not be thrown exception here. LOGGER.info("No file were collected for the table of aware-bucket"); } + return InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -111,15 +117,14 @@ public static DataStream buildSource( Pattern excludingPattern, Pattern databasePattern, Duration partitionIdleTime) { - CombinedAwareBatchSourceFunction function = - new CombinedAwareBatchSourceFunction( + CombinedAwareBatchSource source = + new CombinedAwareBatchSource( catalogLoader, includingPattern, excludingPattern, databasePattern); - StreamSource, ?> sourceOperator = new StreamSource<>(function); TupleTypeInfo> tupleTypeInfo = new TupleTypeInfo<>( new JavaTypeInfo<>(Split.class), BasicTypeInfo.STRING_TYPE_INFO); - return new DataStreamSource<>( - env, tupleTypeInfo, sourceOperator, false, name, Boundedness.BOUNDED) + + return env.fromSource(source, WatermarkStrategy.noWatermarks(), name, tupleTypeInfo) .forceNonParallel() .partitionCustom( (key, numPartitions) -> key % numPartitions, @@ -129,12 +134,4 @@ public static DataStream buildSource( typeInfo, new MultiTablesReadOperator(catalogLoader, false, partitionIdleTime)); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java similarity index 63% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java index 01e0127e9fda..9bd4a84f571c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java @@ -21,21 +21,23 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.table.data.RowData; import java.util.regex.Pattern; @@ -44,13 +46,11 @@ import static org.apache.paimon.flink.compact.MultiTableScanBase.ScanResult.IS_EMPTY; /** It is responsible for monitoring compactor source of multi bucket table in stream mode. */ -public class CombinedAwareStreamingSourceFunction - extends CombinedCompactorSourceFunction> { +public class CombinedAwareStreamingSource extends CombinedCompactorSource> { private final long monitorInterval; - private transient MultiTableScanBase> tableScan; - public CombinedAwareStreamingSourceFunction( + public CombinedAwareStreamingSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -60,39 +60,46 @@ public CombinedAwareStreamingSourceFunction( this.monitorInterval = monitorInterval; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public SourceReader, SimpleSourceSplit> createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiAwareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader extends AbstractNonCoordinatedSourceReader> { + private transient MultiTableScanBase> tableScan; - @SuppressWarnings("BusyWait") - @Override - void scanTable() throws Exception { - while (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public void start() { + super.start(); + tableScan = + new MultiAwareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext(ReaderOutput> readerOutput) + throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -106,37 +113,22 @@ public static DataStream buildSource( Pattern databasePattern, long monitorInterval) { - CombinedAwareStreamingSourceFunction function = - new CombinedAwareStreamingSourceFunction( + CombinedAwareStreamingSource source = + new CombinedAwareStreamingSource( catalogLoader, includingPattern, excludingPattern, databasePattern, monitorInterval); - StreamSource, ?> sourceOperator = new StreamSource<>(function); - boolean isParallel = false; TupleTypeInfo> tupleTypeInfo = new TupleTypeInfo<>( new JavaTypeInfo<>(Split.class), BasicTypeInfo.STRING_TYPE_INFO); - return new DataStreamSource<>( - env, - tupleTypeInfo, - sourceOperator, - isParallel, - name, - Boundedness.CONTINUOUS_UNBOUNDED) + + return env.fromSource(source, WatermarkStrategy.noWatermarks(), name, tupleTypeInfo) .forceNonParallel() .partitionCustom( (key, numPartitions) -> key % numPartitions, split -> ((DataSplit) split.f0).bucket()) .transform(name, typeInfo, new MultiTablesReadOperator(catalogLoader, true)); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java similarity index 63% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java index 02bb8786505d..f58d86cdd65e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java @@ -20,13 +20,11 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.api.connector.source.Boundedness; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; /** @@ -45,8 +43,7 @@ *

    Currently, only dedicated compaction job for multi-tables rely on this monitor. This is the * single (non-parallel) monitoring task, it is responsible for the new Paimon table. */ -public abstract class CombinedCompactorSourceFunction extends RichSourceFunction { - +public abstract class CombinedCompactorSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 2L; protected final Catalog.Loader catalogLoader; @@ -55,10 +52,7 @@ public abstract class CombinedCompactorSourceFunction extends RichSourceFunct protected final Pattern databasePattern; protected final boolean isStreaming; - protected transient AtomicBoolean isRunning; - protected transient SourceContext ctx; - - public CombinedCompactorSourceFunction( + public CombinedCompactorSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -71,37 +65,8 @@ public CombinedCompactorSourceFunction( this.isStreaming = isStreaming; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); - } - - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - isRunning = new AtomicBoolean(true); - } - @Override - public void run(SourceContext sourceContext) throws Exception { - this.ctx = sourceContext; - scanTable(); + public Boundedness getBoundedness() { + return isStreaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning.set(false); - } - } else { - isRunning.set(false); - } - } - - abstract void scanTable() throws Exception; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java similarity index 71% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java index 6a40f10ada61..64f0c38f5a11 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java @@ -25,18 +25,20 @@ import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,14 +57,12 @@ * It is responsible for the batch compactor source of the table with unaware bucket in combined * mode. */ -public class CombinedUnawareBatchSourceFunction - extends CombinedCompactorSourceFunction { +public class CombinedUnawareBatchSource + extends CombinedCompactorSource { - private static final Logger LOGGER = - LoggerFactory.getLogger(CombinedUnawareBatchSourceFunction.class); - private transient MultiTableScanBase tableScan; + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedUnawareBatchSource.class); - public CombinedUnawareBatchSourceFunction( + public CombinedUnawareBatchSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -70,34 +70,34 @@ public CombinedUnawareBatchSourceFunction( super(catalogLoader, includingPattern, excludingPattern, databasePattern, false); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiUnawareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader + extends AbstractNonCoordinatedSourceReader { + private transient MultiTableScanBase tableScan; + + @Override + public void start() { + super.start(); + tableScan = + new MultiUnawareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } - @Override - void scanTable() throws Exception { - if (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public InputStatus pollNext( + ReaderOutput readerOutput) throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { // Currently, in the combined mode, there are two scan tasks for the table of two @@ -106,6 +106,15 @@ void scanTable() throws Exception { // should not be thrown exception here. LOGGER.info("No file were collected for the table of unaware-bucket"); } + return InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -117,22 +126,18 @@ public static DataStream buildSource( Pattern excludingPattern, Pattern databasePattern, @Nullable Duration partitionIdleTime) { - CombinedUnawareBatchSourceFunction function = - new CombinedUnawareBatchSourceFunction( + CombinedUnawareBatchSource combinedUnawareBatchSource = + new CombinedUnawareBatchSource( catalogLoader, includingPattern, excludingPattern, databasePattern); - StreamSource - sourceOperator = new StreamSource<>(function); MultiTableCompactionTaskTypeInfo compactionTaskTypeInfo = new MultiTableCompactionTaskTypeInfo(); SingleOutputStreamOperator source = - new DataStreamSource<>( - env, - compactionTaskTypeInfo, - sourceOperator, - false, + env.fromSource( + combinedUnawareBatchSource, + WatermarkStrategy.noWatermarks(), name, - Boundedness.BOUNDED) + compactionTaskTypeInfo) .forceNonParallel(); if (partitionIdleTime != null) { @@ -177,12 +182,4 @@ private static Long getPartitionInfo( } return partitionInfo.get(partition); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java similarity index 57% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java index b64518a7ef60..6ea1ead4db30 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java @@ -23,14 +23,16 @@ import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import java.util.regex.Pattern; @@ -40,13 +42,12 @@ /** * It is responsible for monitoring compactor source in stream mode for the table of unaware bucket. */ -public class CombinedUnawareStreamingSourceFunction - extends CombinedCompactorSourceFunction { +public class CombinedUnawareStreamingSource + extends CombinedCompactorSource { private final long monitorInterval; - private MultiTableScanBase tableScan; - public CombinedUnawareStreamingSourceFunction( + public CombinedUnawareStreamingSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -56,39 +57,47 @@ public CombinedUnawareStreamingSourceFunction( this.monitorInterval = monitorInterval; } - /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. - */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. - */ - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiUnawareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); - } + private class Reader + extends AbstractNonCoordinatedSourceReader { + private MultiTableScanBase tableScan; - @SuppressWarnings("BusyWait") - @Override - void scanTable() throws Exception { - while (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + @Override + public void start() { + super.start(); + tableScan = + new MultiUnawareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext( + ReaderOutput readerOutput) throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -101,33 +110,18 @@ public static DataStream buildSource( Pattern databasePattern, long monitorInterval) { - CombinedUnawareStreamingSourceFunction function = - new CombinedUnawareStreamingSourceFunction( + CombinedUnawareStreamingSource source = + new CombinedUnawareStreamingSource( catalogLoader, includingPattern, excludingPattern, databasePattern, monitorInterval); - StreamSource - sourceOperator = new StreamSource<>(function); - boolean isParallel = false; MultiTableCompactionTaskTypeInfo compactionTaskTypeInfo = new MultiTableCompactionTaskTypeInfo(); - return new DataStreamSource<>( - env, - compactionTaskTypeInfo, - sourceOperator, - isParallel, - name, - Boundedness.CONTINUOUS_UNBOUNDED) - .forceNonParallel(); - } - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } + return env.fromSource( + source, WatermarkStrategy.noWatermarks(), name, compactionTaskTypeInfo) + .forceNonParallel(); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java new file mode 100644 index 000000000000..4e33376930f8 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -0,0 +1,248 @@ +/* + * 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.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +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.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.io.InputStatus; +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.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 java.util.stream.Collectors; + +import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE; + +/** + * This is the single (non-parallel) monitoring task, it is responsible for: + * + *

      + *
    1. Monitoring snapshots of the Paimon table. + *
    2. Creating the {@link Split splits} corresponding to the incremental files + *
    3. 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: + * + *

      + *
    1. Consumer-id: rely on this source to do aligned snapshot consumption, and ensure that all + * data in a snapshot is consumed within each checkpoint. + *
    2. Snapshot-watermark: when there is no watermark definition, the default Paimon table will + * pass the watermark recorded in the snapshot. + *
    + */ +public class MonitorSource extends AbstractNonCoordinatedSource { + + 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; + + public MonitorSource( + ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { + this.readBuilder = readBuilder; + this.monitorInterval = monitorInterval; + this.emitSnapshotWatermark = emitSnapshotWatermark; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private static final String CHECKPOINT_STATE = "CS"; + private static final String NEXT_SNAPSHOT_STATE = "NSS"; + + private final StreamTableScan scan = readBuilder.newStreamScan(); + private final TreeMap nextSnapshotPerCheckpoint = new TreeMap<>(); + + @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 List snapshotState(long checkpointId) { + List results = new ArrayList<>(); + + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + results.add(new SimpleSourceSplit(CHECKPOINT_STATE + nextSnapshot)); + this.nextSnapshotPerCheckpoint.put(checkpointId, nextSnapshot); + } + + this.nextSnapshotPerCheckpoint.forEach( + (k, v) -> + results.add(new SimpleSourceSplit(NEXT_SNAPSHOT_STATE + k + ":" + v))); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + return results; + } + + @Override + public void addSplits(List list) { + LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + + List retrievedStates = + list.stream() + .map(SimpleSourceSplit::splitId) + .filter(x -> x.startsWith(CHECKPOINT_STATE)) + .map(x -> Long.parseLong(x.substring(CHECKPOINT_STATE.length()))) + .collect(Collectors.toList()); + + // given that the parallelism of the source 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)); + } + + list.stream() + .map(SimpleSourceSplit::splitId) + .filter(x -> x.startsWith(NEXT_SNAPSHOT_STATE)) + .map(x -> x.substring(NEXT_SNAPSHOT_STATE.length()).split(":")) + .forEach( + x -> + nextSnapshotPerCheckpoint.put( + Long.parseLong(x[0]), Long.parseLong(x[1]))); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + boolean isEmpty; + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(readerOutput::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + readerOutput.emitWatermark(new Watermark(watermark)); + } + } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return InputStatus.END_OF_INPUT; + } + + if (isEmpty) { + Thread.sleep(monitorInterval); + } + return InputStatus.MORE_AVAILABLE; + } + } + + public static DataStream buildSource( + StreamExecutionEnvironment env, + String name, + TypeInformation typeInfo, + ReadBuilder readBuilder, + long monitorInterval, + boolean emitSnapshotWatermark, + boolean shuffleBucketWithPartition, + BucketMode bucketMode) { + SingleOutputStreamOperator singleOutputStreamOperator = + env.fromSource( + new MonitorSource( + readBuilder, monitorInterval, emitSnapshotWatermark), + WatermarkStrategy.noWatermarks(), + 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-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java index 73d46ae1e3f1..fbc8bb9d756a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java @@ -52,9 +52,8 @@ /** * The operator that reads the Tuple2<{@link Split}, String> received from the preceding {@link - * CombinedAwareBatchSourceFunction} or {@link CombinedAwareStreamingSourceFunction}. Contrary to - * the {@link CombinedCompactorSourceFunction} which has a parallelism of 1, this operator can have - * DOP > 1. + * CombinedAwareBatchSource} or {@link CombinedAwareStreamingSource}. Contrary to the {@link + * CombinedCompactorSource} which has a parallelism of 1, this operator can have DOP > 1. */ public class MultiTablesReadOperator extends AbstractStreamOperator implements OneInputStreamOperator, RowData> { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java index c501c2519b41..0864741a178f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java @@ -44,7 +44,7 @@ /** * The operator is used for historical partition compaction. It reads {@link * MultiTableUnawareAppendCompactionTask} received from the preceding {@link - * CombinedUnawareBatchSourceFunction} and filter partitions which is not historical. + * CombinedUnawareBatchSource} and filter partitions which is not historical. */ public class MultiUnawareTablesReadOperator extends AbstractStreamOperator diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java index 80c85f7cdb35..6caf4544e514 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java @@ -38,8 +38,8 @@ /** * The operator that reads the {@link Split splits} received from the preceding {@link - * MonitorFunction}. Contrary to the {@link MonitorFunction} which has a parallelism of 1, this - * operator can have DOP > 1. + * MonitorSource}. Contrary to the {@link MonitorSource} which has a parallelism of 1, this operator + * can have DOP > 1. */ public class ReadOperator extends AbstractStreamOperator implements OneInputStreamOperator { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java index 6a2c7b071d2d..5245114e80ee 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java @@ -36,6 +36,7 @@ import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.FailingFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.dag.Transformation; @@ -450,7 +451,12 @@ private void sinkAndValidate( throw new UnsupportedOperationException(); } DataStreamSource source = - env.addSource(new FiniteTestSource<>(src, true), InternalTypeInfo.of(TABLE_TYPE)); + env.fromSource( + new FiniteTestSource<>(src, true), + WatermarkStrategy.noWatermarks(), + "FiniteTestSource", + InternalTypeInfo.of(TABLE_TYPE)); + source.forceNonParallel(); new FlinkSinkBuilder(table).forRowData(source).build(); env.execute(); assertThat(iterator.collect(expected.length)).containsExactlyInAnyOrder(expected); @@ -521,9 +527,13 @@ public static DataStreamSource buildTestSource( StreamExecutionEnvironment env, boolean isBatch) { return isBatch ? env.fromCollection(SOURCE_DATA, InternalTypeInfo.of(TABLE_TYPE)) - : env.addSource( - new FiniteTestSource<>(SOURCE_DATA, false), - InternalTypeInfo.of(TABLE_TYPE)); + : (DataStreamSource) + env.fromSource( + new FiniteTestSource<>(SOURCE_DATA, false), + WatermarkStrategy.noWatermarks(), + "FiniteTestSource", + InternalTypeInfo.of(TABLE_TYPE)) + .forceNonParallel(); } public static List executeAndCollect(DataStream source) throws Exception { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java index 9c5254d6283b..add93bb8b664 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java @@ -18,19 +18,21 @@ package org.apache.paimon.flink; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.utils.Preconditions; -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.typeutils.base.IntSerializer; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; /** * A stream source that: 1) emits a list of elements without allowing checkpoints, 2) then waits for @@ -39,8 +41,7 @@ * *

    The reason this class is rewritten is to support {@link CheckpointedFunction}. */ -public class FiniteTestSource - implements SourceFunction, CheckpointedFunction, CheckpointListener { +public class FiniteTestSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; @@ -48,113 +49,110 @@ public class FiniteTestSource private final boolean emitOnce; - private volatile boolean running = true; - - private transient int numCheckpointsComplete; - - private transient ListState checkpointedState; - - private volatile int numTimesEmitted; - public FiniteTestSource(List elements, boolean emitOnce) { this.elements = elements; this.emitOnce = emitOnce; } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - this.checkpointedState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>("emit-times", IntSerializer.INSTANCE)); - - if (context.isRestored()) { - List retrievedStates = new ArrayList<>(); - for (Integer entry : this.checkpointedState.get()) { - retrievedStates.add(entry); + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext sourceReaderContext) + throws Exception { + return new Reader<>(elements, emitOnce); + } + + private static class Reader extends AbstractNonCoordinatedSourceReader { + + private final List elements; + + private final boolean emitOnce; + + private int numTimesEmitted = 0; + + private int numCheckpointsComplete; + + private Integer checkpointToAwait; + + private Reader(List elements, boolean emitOnce) { + this.elements = elements; + this.emitOnce = emitOnce; + this.numCheckpointsComplete = 0; + } + + @Override + public synchronized InputStatus pollNext(ReaderOutput readerOutput) { + if (checkpointToAwait == null) { + checkpointToAwait = numCheckpointsComplete + 2; } + switch (numTimesEmitted) { + case 0: + emitElements(readerOutput, false); + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + emitElements(readerOutput, true); + if (numCheckpointsComplete < checkpointToAwait + 2) { + return InputStatus.MORE_AVAILABLE; + } + break; + case 1: + emitElements(readerOutput, true); + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + break; + case 2: + // Maybe missed notifyCheckpointComplete, wait next notifyCheckpointComplete + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + break; + } + return InputStatus.END_OF_INPUT; + } + + @Override + public void addSplits(List list) { + List retrievedStates = + list.stream() + .map(x -> Integer.parseInt(x.splitId())) + .collect(Collectors.toList()); // given that the parallelism of the function is 1, we can only have 1 state Preconditions.checkArgument( retrievedStates.size() == 1, getClass().getSimpleName() + " retrieved invalid state."); - this.numTimesEmitted = retrievedStates.get(0); + numTimesEmitted = retrievedStates.get(0); Preconditions.checkArgument( numTimesEmitted <= 2, getClass().getSimpleName() + " retrieved invalid numTimesEmitted: " + numTimesEmitted); - } else { - this.numTimesEmitted = 0; } - } - @Override - public void run(SourceContext ctx) throws Exception { - switch (numTimesEmitted) { - case 0: - emitElementsAndWaitForCheckpoints(ctx, false); - emitElementsAndWaitForCheckpoints(ctx, true); - break; - case 1: - emitElementsAndWaitForCheckpoints(ctx, true); - break; - case 2: - // Maybe missed notifyCheckpointComplete, wait next notifyCheckpointComplete - final Object lock = ctx.getCheckpointLock(); - synchronized (lock) { - int checkpointToAwait = numCheckpointsComplete + 2; - while (running && numCheckpointsComplete < checkpointToAwait) { - lock.wait(1); - } - } - break; + @Override + public List snapshotState(long l) { + return Collections.singletonList( + new SimpleSourceSplit(Integer.toString(numTimesEmitted))); } - } - private void emitElementsAndWaitForCheckpoints(SourceContext ctx, boolean isSecond) - throws InterruptedException { - final Object lock = ctx.getCheckpointLock(); + @Override + public void notifyCheckpointComplete(long checkpointId) { + numCheckpointsComplete++; + } - final int checkpointToAwait; - synchronized (lock) { - checkpointToAwait = numCheckpointsComplete + 2; + private void emitElements(ReaderOutput readerOutput, boolean isSecond) { if (!isSecond || !emitOnce) { for (T t : elements) { - ctx.collect(t); + readerOutput.collect(t); } } numTimesEmitted++; } - - synchronized (lock) { - while (running && numCheckpointsComplete < checkpointToAwait) { - lock.wait(1); - } - } - } - - @Override - public void cancel() { - running = false; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - numCheckpointsComplete++; - } - - @Override - public void notifyCheckpointAborted(long checkpointId) {} - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - Preconditions.checkState( - this.checkpointedState != null, - "The " + getClass().getSimpleName() + " has not been properly initialized."); - - this.checkpointedState.clear(); - this.checkpointedState.add(this.numTimesEmitted); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java index 594affc124eb..75b96cbe02eb 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java @@ -47,8 +47,10 @@ public SerializableRowData(RowData row, TypeSerializer serializer) { this.serializer = serializer; } - private void writeObject(ObjectOutputStream out) throws IOException { + private synchronized void writeObject(ObjectOutputStream out) throws IOException { out.defaultWriteObject(); + // This following invocation needs to be synchronized to avoid racing problems when the + // serializer is reused across multiple subtasks. serializer.serialize(row, new DataOutputViewStreamWrapper(out)); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java index eeb99df75cb5..301849070d6d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java @@ -20,7 +20,9 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.flink.utils.RuntimeContextUtils; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.reader.RecordReader; @@ -29,9 +31,14 @@ import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.utils.FailingFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.types.Row; @@ -372,7 +379,12 @@ public void testStatelessWriter() throws Exception { .checkpointIntervalMs(500) .build(); DataStream source = - env.addSource(new TestStatelessWriterSource(table)).setParallelism(2).forward(); + env.fromSource( + new TestStatelessWriterSource(table), + WatermarkStrategy.noWatermarks(), + "TestStatelessWriterSource") + .setParallelism(2) + .forward(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); tEnv.registerCatalog("mycat", sEnv.getCatalog("PAIMON").get()); @@ -384,46 +396,59 @@ public void testStatelessWriter() throws Exception { .containsExactlyInAnyOrder(Row.of(1, "test"), Row.of(2, "test")); } - private static class TestStatelessWriterSource extends RichParallelSourceFunction { + private static class TestStatelessWriterSource extends AbstractNonCoordinatedSource { private final FileStoreTable table; - private volatile boolean isRunning = true; - private TestStatelessWriterSource(FileStoreTable table) { this.table = table; } @Override - public void run(SourceContext sourceContext) throws Exception { - int taskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); - // wait some time in parallelism #2, - // so that it does not commit in the same checkpoint with parallelism #1 - int waitCount = (taskId == 0 ? 0 : 10); - - while (isRunning) { - synchronized (sourceContext.getCheckpointLock()) { - if (taskId == 0) { + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(sourceReaderContext.getIndexOfSubtask()); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private final int taskId; + private int waitCount; + + private Reader(int taskId) { + this.taskId = taskId; + this.waitCount = (taskId == 0 ? 0 : 10); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + if (taskId == 0) { + if (waitCount == 0) { + readerOutput.collect(1); + } else if (countNumRecords() >= 1) { + // wait for the record to commit before exiting + Thread.sleep(1000); + return InputStatus.END_OF_INPUT; + } + } else { + int numRecords = countNumRecords(); + if (numRecords >= 1) { if (waitCount == 0) { - sourceContext.collect(1); - } else if (countNumRecords() >= 1) { - // wait for the record to commit before exiting - break; - } - } else { - int numRecords = countNumRecords(); - if (numRecords >= 1) { - if (waitCount == 0) { - sourceContext.collect(2); - } else if (countNumRecords() >= 2) { - // make sure the next checkpoint is successful - break; - } + readerOutput.collect(2); + } else if (countNumRecords() >= 2) { + // make sure the next checkpoint is successful + Thread.sleep(1000); + return InputStatus.END_OF_INPUT; } } - waitCount--; } + waitCount--; Thread.sleep(1000); + return InputStatus.MORE_AVAILABLE; } } @@ -439,11 +464,6 @@ private int countNumRecords() throws Exception { } return ret; } - - @Override - public void cancel() { - isRunning = false; - } } @Override diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java index 61a03a29a21b..b1e0fb83610e 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java @@ -33,12 +33,17 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.types.DataTypes; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.GenericRowData; @@ -46,6 +51,7 @@ import org.apache.flink.table.runtime.typeutils.InternalSerializers; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.function.SupplierWithException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -58,11 +64,13 @@ import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.paimon.CoreOptions.CONSUMER_ID; import static org.assertj.core.api.Assertions.assertThat; -/** Test for {@link MonitorFunction} and {@link ReadOperator}. */ +/** Test for {@link MonitorSource} and {@link ReadOperator}. */ public class OperatorSourceTest { @TempDir Path tempDir; @@ -114,28 +122,39 @@ private List> readSplit(Split split) throws IOException { } @Test - public void testMonitorFunction() throws Exception { + public void testMonitorSource() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // 1. run first OperatorSubtaskState snapshot; { - MonitorFunction function = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource src = new StreamSource<>(function); + MonitorSource source = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operator = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + source.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarness = - new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operator, 1, 1, 0); testHarness.open(); - snapshot = testReadSplit(function, () -> testHarness.snapshot(0, 0), 1, 1, 1); + snapshot = testReadSplit(operator, () -> testHarness.snapshot(0, 0), 1, 1, 1); } // 2. restore from state { - MonitorFunction functionCopy1 = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource srcCopy1 = new StreamSource<>(functionCopy1); + MonitorSource sourceCopy1 = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operatorCopy1 = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + sourceCopy1.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarnessCopy1 = - new AbstractStreamOperatorTestHarness<>(srcCopy1, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operatorCopy1, 1, 1, 0); testHarnessCopy1.initializeState(snapshot); testHarnessCopy1.open(); testReadSplit( - functionCopy1, + operatorCopy1, () -> { testHarnessCopy1.snapshot(1, 1); testHarnessCopy1.notifyOfCompletedCheckpoint(1); @@ -148,12 +167,17 @@ public void testMonitorFunction() throws Exception { // 3. restore from consumer id { - MonitorFunction functionCopy2 = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource srcCopy2 = new StreamSource<>(functionCopy2); + MonitorSource sourceCopy2 = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operatorCopy2 = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + sourceCopy2.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarnessCopy2 = - new AbstractStreamOperatorTestHarness<>(srcCopy2, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operatorCopy2, 1, 1, 0); testHarnessCopy2.open(); - testReadSplit(functionCopy2, () -> null, 3, 3, 3); + testReadSplit(operatorCopy2, () -> null, 3, 3, 3); } } @@ -231,7 +255,7 @@ public void testReadOperatorMetricsRegisterAndUpdate() throws Exception { } private T testReadSplit( - MonitorFunction function, + SourceOperator operator, SupplierWithException beforeClose, int a, int b, @@ -239,20 +263,36 @@ private T testReadSplit( throws Exception { Throwable[] error = new Throwable[1]; ArrayBlockingQueue queue = new ArrayBlockingQueue<>(10); + AtomicReference> iteratorRef = new AtomicReference<>(); - DummySourceContext sourceContext = - new DummySourceContext() { + PushingAsyncDataInput.DataOutput output = + new PushingAsyncDataInput.DataOutput() { @Override - public void collect(Split element) { - queue.add(element); + public void emitRecord(StreamRecord streamRecord) { + queue.add(streamRecord.getValue()); } + + @Override + public void emitWatermark(Watermark watermark) {} + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) {} + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) {} }; + AtomicBoolean isRunning = new AtomicBoolean(true); Thread runner = new Thread( () -> { try { - function.run(sourceContext); + while (isRunning.get()) { + operator.emitNext(output); + } } catch (Throwable t) { t.printStackTrace(); error[0] = t; @@ -266,34 +306,15 @@ public void collect(Split element) { assertThat(readSplit(split)).containsExactlyInAnyOrder(Arrays.asList(a, b, c)); T t = beforeClose.get(); - function.cancel(); + CloseableIterator iterator = iteratorRef.get(); + if (iterator != null) { + iterator.close(); + } + isRunning.set(false); runner.join(); assertThat(error[0]).isNull(); return t; } - - private abstract static class DummySourceContext - implements SourceFunction.SourceContext { - - private final Object lock = new Object(); - - @Override - public void collectWithTimestamp(Split element, long timestamp) {} - - @Override - public void emitWatermark(Watermark mark) {} - - @Override - public void markAsTemporarilyIdle() {} - - @Override - public Object getCheckpointLock() { - return lock; - } - - @Override - public void close() {} - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java new file mode 100644 index 000000000000..8e24db54c5ec --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java @@ -0,0 +1,175 @@ +/* + * 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.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SimpleSourceSplitSerializer; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.api.operators.SourceOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.MockStreamingRuntimeContext; + +import java.util.ArrayList; +import java.util.Collections; + +/** A SourceOperator extension to simplify test setup. */ +public class TestingSourceOperator extends SourceOperator { + + private static final long serialVersionUID = 1L; + + private final int subtaskIndex; + private final int parallelism; + + public TestingSourceOperator( + StreamOperatorParameters parameters, + SourceReader reader, + WatermarkStrategy watermarkStrategy, + ProcessingTimeService timeService, + boolean emitProgressiveWatermarks) { + + this( + parameters, + reader, + watermarkStrategy, + timeService, + new TestingOperatorEventGateway(), + 1, + 5, + emitProgressiveWatermarks); + } + + public TestingSourceOperator( + StreamOperatorParameters parameters, + SourceReader reader, + WatermarkStrategy watermarkStrategy, + ProcessingTimeService timeService, + OperatorEventGateway eventGateway, + int subtaskIndex, + int parallelism, + boolean emitProgressiveWatermarks) { + + super( + (context) -> reader, + eventGateway, + new SimpleSourceSplitSerializer(), + watermarkStrategy, + timeService, + new Configuration(), + "localhost", + emitProgressiveWatermarks, + () -> false); + + this.subtaskIndex = subtaskIndex; + this.parallelism = parallelism; + this.metrics = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup(); + initSourceMetricGroup(); + + // unchecked wrapping is okay to keep tests simpler + try { + initReader(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + } + + @Override + public StreamingRuntimeContext getRuntimeContext() { + return new MockStreamingRuntimeContext(false, parallelism, subtaskIndex); + } + + // this is overridden to avoid complex mock injection through the "containingTask" + @Override + public ExecutionConfig getExecutionConfig() { + ExecutionConfig cfg = new ExecutionConfig(); + cfg.setAutoWatermarkInterval(100); + return cfg; + } + + public static SourceOperator createTestOperator( + SourceReader reader, + WatermarkStrategy watermarkStrategy, + boolean emitProgressiveWatermarks) + throws Exception { + + AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); + Environment env = new MockEnvironmentBuilder().build(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + final OperatorStateStore operatorStateStore = + abstractStateBackend.createOperatorStateBackend( + new OperatorStateBackendParametersImpl( + env, + "test-operator", + Collections.emptyList(), + cancelStreamRegistry)); + + final StateInitializationContext stateContext = + new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + + TestProcessingTimeService timeService = new TestProcessingTimeService(); + timeService.setCurrentTime(Integer.MAX_VALUE); // start somewhere that is not zero + + final SourceOperator sourceOperator = + new TestingSourceOperator<>( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask(new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null), + reader, + watermarkStrategy, + timeService, + emitProgressiveWatermarks); + sourceOperator.initializeState(stateContext); + sourceOperator.open(); + + return sourceOperator; + } + + private static class TestingOperatorEventGateway implements OperatorEventGateway { + @Override + public void sendEventToCoordinator(OperatorEvent event) {} + } +} From 0259a7e5c25c192af47efd8006e9aa9564990e03 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Sat, 23 Nov 2024 15:23:57 +0800 Subject: [PATCH 16/17] [flink] Avoid relying on format of table description --- .../paimon/flink/CatalogTableITCase.java | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java index 2a855796d8d4..96334de3f87b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java @@ -251,17 +251,20 @@ public void testSchemasTable() { sql("ALTER TABLE T SET ('snapshot.num-retained.min' = '18')"); sql("ALTER TABLE T SET ('manifest.format' = 'avro')"); - assertThat(sql("SHOW CREATE TABLE T$schemas").toString()) - .isEqualTo( - "[+I[CREATE TABLE `PAIMON`.`default`.`T$schemas` (\n" - + " `schema_id` BIGINT NOT NULL,\n" - + " `fields` VARCHAR(2147483647) NOT NULL,\n" - + " `partition_keys` VARCHAR(2147483647) NOT NULL,\n" - + " `primary_keys` VARCHAR(2147483647) NOT NULL,\n" - + " `options` VARCHAR(2147483647) NOT NULL,\n" - + " `comment` VARCHAR(2147483647),\n" - + " `update_time` TIMESTAMP(3) NOT NULL\n" - + ") ]]"); + String actualResult = sql("SHOW CREATE TABLE T$schemas").toString(); + String expectedResult = + "[+I[CREATE TABLE `PAIMON`.`default`.`T$schemas` (\n" + + " `schema_id` BIGINT NOT NULL,\n" + + " `fields` VARCHAR(2147483647) NOT NULL,\n" + + " `partition_keys` VARCHAR(2147483647) NOT NULL,\n" + + " `primary_keys` VARCHAR(2147483647) NOT NULL,\n" + + " `options` VARCHAR(2147483647) NOT NULL,\n" + + " `comment` VARCHAR(2147483647),\n" + + " `update_time` TIMESTAMP(3) NOT NULL\n" + + ") ]]"; + actualResult = actualResult.replace(" ", "").replace("\n", ""); + expectedResult = expectedResult.replace(" ", "").replace("\n", ""); + assertThat(actualResult).isEqualTo(expectedResult); List result = sql( From 35b057d0877552f212bd3addc51e3116a75a6c72 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Sat, 23 Nov 2024 15:23:57 +0800 Subject: [PATCH 17/17] [flink] Avoid deprecated sql syntax --- .../paimon/flink/PreAggregationITCase.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PreAggregationITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PreAggregationITCase.java index b8dfd8f6a86e..e79b30629087 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PreAggregationITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PreAggregationITCase.java @@ -154,14 +154,14 @@ public void testMergeInMemory() { // So we need to sort the input data. batchSql( "CREATE TABLE myTable AS " - + "SELECT b, c, d FROM " + + "SELECT a, b, c, d FROM " + "(VALUES " + " (1, 1, 2, 'first line')," + " (2, 1, 2, CAST(NULL AS STRING))," + " (3, 1, 2, 'second line')" + ") AS V(a, b, c, d) " + "ORDER BY a"); - batchSql("INSERT INTO T6 SELECT * FROM myTable"); + batchSql("INSERT INTO T6 SELECT b, c, d FROM myTable ORDER BY a"); List result = batchSql("SELECT * FROM T6"); assertThat(result).containsExactlyInAnyOrder(Row.of(1, 2, "first line,second line")); } @@ -232,14 +232,14 @@ public void testMergeInMemory() { // So we need to sort the input data. batchSql( "CREATE TABLE myTable AS " - + "SELECT b, c, d, e FROM " + + "SELECT a, b, c, d, e FROM " + "(VALUES " + " (1, 1, 2, CAST(NULL AS INT), CAST('2020-01-01' AS DATE))," + " (2, 1, 2, 2, CAST('2020-01-02' AS DATE))," + " (3, 1, 2, 3, CAST(NULL AS DATE))" + ") AS V(a, b, c, d, e) " + "ORDER BY a"); - batchSql("INSERT INTO T5 SELECT * FROM myTable"); + batchSql("INSERT INTO T5 SELECT b, c, d, e FROM myTable ORDER BY a"); List result = batchSql("SELECT * FROM T5"); assertThat(result).containsExactlyInAnyOrder(Row.of(1, 2, 3, null)); } @@ -311,14 +311,14 @@ public void testMergeInMemory() { // So we need to sort the input data. batchSql( "CREATE TABLE myTable AS " - + "SELECT b, c, d, e, f FROM " + + "SELECT a, b, c, d, e, f FROM " + "(VALUES " + " (1, 1, 2, CAST(NULL AS INT), 4, CAST('2020-01-01' AS DATE))," + " (2, 1, 2, 2, CAST(NULL as INT), CAST('2020-01-02' AS DATE))," + " (3, 1, 2, 3, 5, CAST(NULL AS DATE))" + ") AS V(a, b, c, d, e, f) " + "ORDER BY a"); - batchSql("INSERT INTO T4 SELECT * FROM myTable"); + batchSql("INSERT INTO T4 SELECT b, c, d, e, f FROM myTable ORDER BY a"); List result = batchSql("SELECT * FROM T4"); assertThat(result) .containsExactlyInAnyOrder(Row.of(1, 2, 3, 5, LocalDate.of(2020, 1, 2))); @@ -1807,14 +1807,15 @@ public void testMergeInMemory() { // So we need to sort the input data. batchSql( "CREATE TABLE myTable AS " - + "SELECT b, c, d, e, f FROM " + + "SELECT a, b, c, d, e, f FROM " + "(VALUES " + " (1, 1, 2, CAST(NULL AS INT), 4, CAST('2020-01-01' AS DATE))," + " (2, 1, 2, 2, CAST(NULL as INT), CAST('2020-01-02' AS DATE))," + " (3, 1, 2, 3, 5, CAST(NULL AS DATE))" + ") AS V(a, b, c, d, e, f) " + "ORDER BY a"); - batchSql("INSERT INTO test_default_agg_func SELECT * FROM myTable"); + batchSql( + "INSERT INTO test_default_agg_func SELECT b, c, d, e, f FROM myTable ORDER BY a"); List result = batchSql("SELECT * FROM test_default_agg_func"); assertThat(result) .containsExactlyInAnyOrder(Row.of(1, 2, 2, 4, LocalDate.of(2020, 1, 2)));