This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git
The following commit(s) were added to refs/heads/master by this push:
new c57847f88f [flink] Shuffle record by primary keys when writing into
postpone bucket tables (#5614)
c57847f88f is described below
commit c57847f88f88e7df45d389ffc1a081a7e1f04f5d
Author: tsreaper <[email protected]>
AuthorDate: Fri May 16 17:37:35 2025 +0800
[flink] Shuffle record by primary keys when writing into postpone bucket
tables (#5614)
---
.../apache/paimon/flink/sink/FlinkSinkBuilder.java | 7 ++-
.../flink/sink/PostponeBucketChannelComputer.java | 56 ++++++++++++++++++++++
.../paimon/flink/PostponeBucketTableITCase.java | 34 +++++++++++++
3 files changed, 93 insertions(+), 4 deletions(-)
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java
index d037825c85..6abb9c47da 100644
---
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java
@@ -56,7 +56,6 @@ import java.util.Map;
import static org.apache.paimon.CoreOptions.OrderType.HILBERT;
import static org.apache.paimon.CoreOptions.OrderType.ORDER;
import static org.apache.paimon.CoreOptions.OrderType.ZORDER;
-import static org.apache.paimon.CoreOptions.createCommitUser;
import static
org.apache.paimon.flink.FlinkConnectorOptions.CLUSTERING_SAMPLE_FACTOR;
import static
org.apache.paimon.flink.FlinkConnectorOptions.CLUSTERING_STRATEGY;
import static
org.apache.paimon.flink.FlinkConnectorOptions.MIN_CLUSTERING_SAMPLE_FACTOR;
@@ -296,10 +295,10 @@ public class FlinkSinkBuilder {
}
private DataStreamSink<?> buildPostponeBucketSink(DataStream<InternalRow>
input) {
+ DataStream<InternalRow> partitioned =
+ partition(input, new
PostponeBucketChannelComputer(table.schema()), parallelism);
FixedBucketSink sink = new FixedBucketSink(table, overwritePartition,
null);
- String commitUser =
createCommitUser(table.coreOptions().toConfiguration());
- DataStream<Committable> written = sink.doWrite(input, commitUser,
parallelism);
- return sink.doCommit(written, commitUser);
+ return sink.sinkFrom(partitioned);
}
private DataStreamSink<?> buildUnawareBucketSink(DataStream<InternalRow>
input) {
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBucketChannelComputer.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBucketChannelComputer.java
new file mode 100644
index 0000000000..ca2bc17a0e
--- /dev/null
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PostponeBucketChannelComputer.java
@@ -0,0 +1,56 @@
+/*
+ * 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.data.InternalRow;
+import org.apache.paimon.schema.TableSchema;
+import org.apache.paimon.table.sink.ChannelComputer;
+import org.apache.paimon.table.sink.KeyAndBucketExtractor;
+import org.apache.paimon.table.sink.PostponeBucketRowKeyExtractor;
+
+/**
+ * {@link ChannelComputer} for writing {@link InternalRow}s into postpone
bucket tables. Records
+ * with same primary keys are distributed to the same subtask.
+ */
+public class PostponeBucketChannelComputer implements
ChannelComputer<InternalRow> {
+
+ private static final long serialVersionUID = 1L;
+
+ private final TableSchema schema;
+
+ private transient int numChannels;
+ private transient KeyAndBucketExtractor<InternalRow> extractor;
+
+ public PostponeBucketChannelComputer(TableSchema schema) {
+ this.schema = schema;
+ }
+
+ @Override
+ public void setup(int numChannels) {
+ this.numChannels = numChannels;
+ this.extractor = new PostponeBucketRowKeyExtractor(schema);
+ }
+
+ @Override
+ public int channel(InternalRow record) {
+ extractor.setRecord(record);
+ return Math.abs(extractor.partition().hashCode() +
extractor.trimmedPrimaryKey().hashCode())
+ % numChannels;
+ }
+}
diff --git
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PostponeBucketTableITCase.java
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PostponeBucketTableITCase.java
index adcd76c469..694ba66e50 100644
---
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PostponeBucketTableITCase.java
+++
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PostponeBucketTableITCase.java
@@ -595,6 +595,40 @@ public class PostponeBucketTableITCase extends
AbstractTestBase {
"+I[1, 11]", "+I[2, 52]", "+I[3, 32]", "+I[4, 40]",
"+I[5, 51]");
}
+ @Test
+ public void testSameKeyPreserveOrder() throws Exception {
+ String warehouse = getTempDirPath();
+ TableEnvironment tEnv =
+ tableEnvironmentBuilder()
+ .batchMode()
+ .setConf(TableConfigOptions.TABLE_DML_SYNC, true)
+ .build();
+
+ tEnv.executeSql(
+ "CREATE CATALOG mycat WITH (\n"
+ + " 'type' = 'paimon',\n"
+ + " 'warehouse' = '"
+ + warehouse
+ + "'\n"
+ + ")");
+ tEnv.executeSql("USE CATALOG mycat");
+ tEnv.executeSql(
+ "CREATE TABLE T (\n"
+ + " k INT,\n"
+ + " v INT,\n"
+ + " PRIMARY KEY (k) NOT ENFORCED\n"
+ + ") WITH (\n"
+ + " 'bucket' = '-2'\n"
+ + ")");
+
+ tEnv.executeSql(
+ "INSERT INTO T /*+ OPTIONS('sink.parallelism' = '2')
*/ VALUES (1, 10), (1, 20), (1, 30), (1, 40)")
+ .await();
+ tEnv.executeSql("CALL sys.compact(`table` => 'default.T')").await();
+ assertThat(collect(tEnv.executeSql("SELECT * FROM T")))
+ .containsExactlyInAnyOrder("+I[1, 40]");
+ }
+
private List<String> collect(TableResult result) throws Exception {
List<String> ret = new ArrayList<>();
try (CloseableIterator<Row> it = result.collect()) {