You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/11/08 12:26:11 UTC

[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #361: [FLINK-29752] Modify Flink Table Store connector to trigger full compaction constantly when full changelog is needed

JingsongLi commented on code in PR #361:
URL: https://github.com/apache/flink-table-store/pull/361#discussion_r1016532970


##########
docs/layouts/shortcodes/generated/core_configuration.html:
##########
@@ -32,6 +32,12 @@
             <td><p>Enum</p></td>
             <td>Whether to double write to a changelog file. This changelog file keeps the details of data changes, it can be read directly during stream reads.<br /><br />Possible values:<ul><li>"none": No changelog file.</li><li>"input": Double write to a changelog file when flushing memory table, the changelog is from input.</li><li>"full-compaction": Generate changelog files with each full compaction.</li></ul></td>
         </tr>
+        <tr>
+            <td><h5>changelog-producer.full-compaction.trigger-interval</h5></td>

Review Comment:
   I think we can make this key shorter.
   Maybe just `changelog-producer.compaction-interval`?



##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/FullChangelogStoreWriteOperator.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.table.store.connector.sink;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.Snapshot;
+import org.apache.flink.table.store.file.utils.SnapshotManager;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.table.sink.LogSinkFunction;
+import org.apache.flink.table.store.table.sink.SinkRecord;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * A {@link StoreWriteOperator} for {@link
+ * org.apache.flink.table.store.CoreOptions.ChangelogProducer#FULL_COMPACTION} changelog producer.
+ */
+public class FullChangelogStoreWriteOperator extends StoreWriteOperator {
+

Review Comment:
   lack serid.



##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/source/FileStoreSource.java:
##########
@@ -90,7 +92,14 @@ public SplitEnumerator<FileStoreSourceSplit, PendingSplitsCheckpoint> restoreEnu
                 snapshotId = snapshotManager.latestSnapshotId();
                 splits = new ArrayList<>();
             } else {
-                DataTableScan.DataFilePlan plan = scan.plan();
+                DataTableScan.DataFilePlan plan;
+                if (table.options().changelogProducer()
+                                == CoreOptions.ChangelogProducer.FULL_COMPACTION
+                        && isContinuous) {
+                    plan = scan.withLevel(table.options().numLevels() - 1).plan();

Review Comment:
   add comments



##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/FullChangelogStoreWriteOperator.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.table.store.connector.sink;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.Snapshot;
+import org.apache.flink.table.store.file.utils.SnapshotManager;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.table.sink.LogSinkFunction;
+import org.apache.flink.table.store.table.sink.SinkRecord;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * A {@link StoreWriteOperator} for {@link
+ * org.apache.flink.table.store.CoreOptions.ChangelogProducer#FULL_COMPACTION} changelog producer.
+ */
+public class FullChangelogStoreWriteOperator extends StoreWriteOperator {
+
+    private final long checkpointThreshold;
+
+    private transient Set<Tuple2<BinaryRowData, Integer>> currentWrittenBuckets;
+    private transient NavigableMap<Long, Set<Tuple2<BinaryRowData, Integer>>> writtenBuckets;
+    private transient ListState<Tuple3<Long, BinaryRowData, Integer>> writtenBucketState;
+    private transient Long snapshotIdentifierToCheck;
+
+    public FullChangelogStoreWriteOperator(
+            FileStoreTable table,
+            String initialCommitUser,
+            @Nullable Map<String, String> overwritePartition,
+            @Nullable LogSinkFunction logSinkFunction,
+            long checkpointThreshold) {
+        super(table, initialCommitUser, overwritePartition, logSinkFunction);
+        this.checkpointThreshold = checkpointThreshold;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        super.initializeState(context);
+
+        TupleSerializer<Tuple3<Long, BinaryRowData, Integer>> writtenBucketStateSerializer =
+                new TupleSerializer<>(
+                        (Class<Tuple3<Long, BinaryRowData, Integer>>) (Class<?>) Tuple3.class,
+                        new TypeSerializer[] {
+                            LongSerializer.INSTANCE,
+                            new BinaryRowDataSerializer(
+                                    table.schema().logicalPartitionType().getFieldCount()),
+                            IntSerializer.INSTANCE
+                        });
+        writtenBucketState =
+                context.getOperatorStateStore()
+                        .getListState(
+                                new ListStateDescriptor<>(
+                                        "table_store_written_buckets",
+                                        writtenBucketStateSerializer));
+        writtenBuckets = new TreeMap<>();
+        writtenBucketState
+                .get()
+                .forEach(
+                        t ->
+                                writtenBuckets
+                                        .computeIfAbsent(t.f0, k -> new HashSet<>())
+                                        .add(Tuple2.of(t.f1, t.f2)));
+
+        snapshotIdentifierToCheck = null;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+        currentWrittenBuckets = new HashSet<>();
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception {
+        SinkRecord record = writeRecord(element);
+
+        BinaryRowData partition = record.partition();
+        int bucket = record.bucket();
+        // partition is a reused BinaryRowData
+        // we first check if the tuple exists to minimize copying
+        if (!currentWrittenBuckets.contains(Tuple2.of(partition, bucket))) {
+            currentWrittenBuckets.add(Tuple2.of(partition.copy(), bucket));
+        }
+    }
+
+    @Override
+    public void snapshotState(StateSnapshotContext context) throws Exception {
+        super.snapshotState(context);
+
+        List<Tuple3<Long, BinaryRowData, Integer>> writtenBucketList = new ArrayList<>();
+        for (Map.Entry<Long, Set<Tuple2<BinaryRowData, Integer>>> entry :
+                writtenBuckets.entrySet()) {
+            for (Tuple2<BinaryRowData, Integer> bucket : entry.getValue()) {
+                writtenBucketList.add(Tuple3.of(entry.getKey(), bucket.f0, bucket.f1));
+            }
+        }
+        writtenBucketState.update(writtenBucketList);
+    }
+
+    @Override
+    protected List<Committable> prepareCommit(boolean blocking, long checkpointId)
+            throws IOException {
+        if (snapshotIdentifierToCheck != null) {
+            Optional<Snapshot> snapshot = findSnapshot(snapshotIdentifierToCheck);
+            if (snapshot.map(s -> s.commitKind() == Snapshot.CommitKind.COMPACT).orElse(false)) {
+                writtenBuckets.headMap(snapshotIdentifierToCheck, true).clear();
+                snapshotIdentifierToCheck = null;
+            }
+        }
+
+        if (!currentWrittenBuckets.isEmpty()) {
+            writtenBuckets
+                    .computeIfAbsent(checkpointId, k -> new HashSet<>())
+                    .addAll(currentWrittenBuckets);
+            currentWrittenBuckets.clear();
+        }
+
+        if (snapshotIdentifierToCheck == null // wait for last forced full compaction to complete
+                && !writtenBuckets.isEmpty() // there should be something to compact
+                && checkpointId - writtenBuckets.navigableKeySet().first() + 1
+                        >= checkpointThreshold // checkpoints without full compaction exceeds
+        ) {
+            blocking = true;

Review Comment:
   blocking => doCompaction?



##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreSink.java:
##########
@@ -85,10 +84,25 @@ public StoreSink(
     }
 
     private OneInputStreamOperator<RowData, Committable> createWriteOperator(
-            String initialCommitUser) {
+            CheckpointConfig checkpointConfig, String initialCommitUser) {
         if (compactionTask) {
             return new StoreCompactOperator(table, initialCommitUser, compactPartitionSpec);
         }
+
+        if (table.options().changelogProducer() == CoreOptions.ChangelogProducer.FULL_COMPACTION) {
+            long checkpointInterval = checkpointConfig.getCheckpointInterval();
+            long checkpointThreshold =
+                    table.options().changelogProducerFullCompactionTriggerInterval().toMillis()
+                            / checkpointInterval;

Review Comment:
   I don't think it's good to bind to checkpoint interval. Checkpoints may be random:
   - For example, some checkpoints will be skipped in case of failure.
   - For example, there can be checkpoint triggered by source in the future.
   
   It would be better to judge directly by time.



##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/FullChangelogStoreWriteOperator.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.table.store.connector.sink;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.Snapshot;
+import org.apache.flink.table.store.file.utils.SnapshotManager;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.table.sink.LogSinkFunction;
+import org.apache.flink.table.store.table.sink.SinkRecord;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * A {@link StoreWriteOperator} for {@link
+ * org.apache.flink.table.store.CoreOptions.ChangelogProducer#FULL_COMPACTION} changelog producer.
+ */
+public class FullChangelogStoreWriteOperator extends StoreWriteOperator {
+
+    private final long checkpointThreshold;
+
+    private transient Set<Tuple2<BinaryRowData, Integer>> currentWrittenBuckets;
+    private transient NavigableMap<Long, Set<Tuple2<BinaryRowData, Integer>>> writtenBuckets;
+    private transient ListState<Tuple3<Long, BinaryRowData, Integer>> writtenBucketState;
+    private transient Long snapshotIdentifierToCheck;
+
+    public FullChangelogStoreWriteOperator(
+            FileStoreTable table,
+            String initialCommitUser,
+            @Nullable Map<String, String> overwritePartition,
+            @Nullable LogSinkFunction logSinkFunction,
+            long checkpointThreshold) {
+        super(table, initialCommitUser, overwritePartition, logSinkFunction);
+        this.checkpointThreshold = checkpointThreshold;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        super.initializeState(context);
+
+        TupleSerializer<Tuple3<Long, BinaryRowData, Integer>> writtenBucketStateSerializer =
+                new TupleSerializer<>(
+                        (Class<Tuple3<Long, BinaryRowData, Integer>>) (Class<?>) Tuple3.class,
+                        new TypeSerializer[] {
+                            LongSerializer.INSTANCE,
+                            new BinaryRowDataSerializer(
+                                    table.schema().logicalPartitionType().getFieldCount()),
+                            IntSerializer.INSTANCE
+                        });
+        writtenBucketState =
+                context.getOperatorStateStore()
+                        .getListState(
+                                new ListStateDescriptor<>(
+                                        "table_store_written_buckets",
+                                        writtenBucketStateSerializer));
+        writtenBuckets = new TreeMap<>();
+        writtenBucketState
+                .get()
+                .forEach(
+                        t ->
+                                writtenBuckets
+                                        .computeIfAbsent(t.f0, k -> new HashSet<>())
+                                        .add(Tuple2.of(t.f1, t.f2)));
+
+        snapshotIdentifierToCheck = null;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+        currentWrittenBuckets = new HashSet<>();
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception {
+        SinkRecord record = writeRecord(element);
+
+        BinaryRowData partition = record.partition();
+        int bucket = record.bucket();
+        // partition is a reused BinaryRowData
+        // we first check if the tuple exists to minimize copying
+        if (!currentWrittenBuckets.contains(Tuple2.of(partition, bucket))) {
+            currentWrittenBuckets.add(Tuple2.of(partition.copy(), bucket));
+        }
+    }
+
+    @Override
+    public void snapshotState(StateSnapshotContext context) throws Exception {
+        super.snapshotState(context);
+
+        List<Tuple3<Long, BinaryRowData, Integer>> writtenBucketList = new ArrayList<>();
+        for (Map.Entry<Long, Set<Tuple2<BinaryRowData, Integer>>> entry :
+                writtenBuckets.entrySet()) {
+            for (Tuple2<BinaryRowData, Integer> bucket : entry.getValue()) {
+                writtenBucketList.add(Tuple3.of(entry.getKey(), bucket.f0, bucket.f1));
+            }
+        }
+        writtenBucketState.update(writtenBucketList);
+    }
+
+    @Override
+    protected List<Committable> prepareCommit(boolean blocking, long checkpointId)
+            throws IOException {
+        if (snapshotIdentifierToCheck != null) {
+            Optional<Snapshot> snapshot = findSnapshot(snapshotIdentifierToCheck);
+            if (snapshot.map(s -> s.commitKind() == Snapshot.CommitKind.COMPACT).orElse(false)) {
+                writtenBuckets.headMap(snapshotIdentifierToCheck, true).clear();
+                snapshotIdentifierToCheck = null;
+            }
+        }
+
+        if (!currentWrittenBuckets.isEmpty()) {
+            writtenBuckets
+                    .computeIfAbsent(checkpointId, k -> new HashSet<>())
+                    .addAll(currentWrittenBuckets);
+            currentWrittenBuckets.clear();
+        }
+
+        if (snapshotIdentifierToCheck == null // wait for last forced full compaction to complete
+                && !writtenBuckets.isEmpty() // there should be something to compact
+                && checkpointId - writtenBuckets.navigableKeySet().first() + 1
+                        >= checkpointThreshold // checkpoints without full compaction exceeds
+        ) {
+            blocking = true;
+        }
+
+        if (blocking) {
+            snapshotIdentifierToCheck = checkpointId;
+            Set<Tuple2<BinaryRowData, Integer>> compactedBuckets = new HashSet<>();
+            for (Set<Tuple2<BinaryRowData, Integer>> buckets : writtenBuckets.values()) {
+                for (Tuple2<BinaryRowData, Integer> bucket : buckets) {
+                    if (compactedBuckets.contains(bucket)) {

Review Comment:
   Why we need a `compactedBuckets`? Deduplicate? Why?



##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/FullChangelogStoreWriteOperator.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.table.store.connector.sink;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.Snapshot;
+import org.apache.flink.table.store.file.utils.SnapshotManager;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.table.sink.LogSinkFunction;
+import org.apache.flink.table.store.table.sink.SinkRecord;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * A {@link StoreWriteOperator} for {@link
+ * org.apache.flink.table.store.CoreOptions.ChangelogProducer#FULL_COMPACTION} changelog producer.
+ */
+public class FullChangelogStoreWriteOperator extends StoreWriteOperator {
+
+    private final long checkpointThreshold;
+
+    private transient Set<Tuple2<BinaryRowData, Integer>> currentWrittenBuckets;
+    private transient NavigableMap<Long, Set<Tuple2<BinaryRowData, Integer>>> writtenBuckets;
+    private transient ListState<Tuple3<Long, BinaryRowData, Integer>> writtenBucketState;
+    private transient Long snapshotIdentifierToCheck;
+
+    public FullChangelogStoreWriteOperator(
+            FileStoreTable table,
+            String initialCommitUser,
+            @Nullable Map<String, String> overwritePartition,
+            @Nullable LogSinkFunction logSinkFunction,
+            long checkpointThreshold) {
+        super(table, initialCommitUser, overwritePartition, logSinkFunction);
+        this.checkpointThreshold = checkpointThreshold;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        super.initializeState(context);
+
+        TupleSerializer<Tuple3<Long, BinaryRowData, Integer>> writtenBucketStateSerializer =
+                new TupleSerializer<>(
+                        (Class<Tuple3<Long, BinaryRowData, Integer>>) (Class<?>) Tuple3.class,
+                        new TypeSerializer[] {
+                            LongSerializer.INSTANCE,
+                            new BinaryRowDataSerializer(
+                                    table.schema().logicalPartitionType().getFieldCount()),
+                            IntSerializer.INSTANCE
+                        });
+        writtenBucketState =
+                context.getOperatorStateStore()
+                        .getListState(
+                                new ListStateDescriptor<>(
+                                        "table_store_written_buckets",
+                                        writtenBucketStateSerializer));
+        writtenBuckets = new TreeMap<>();
+        writtenBucketState
+                .get()
+                .forEach(
+                        t ->
+                                writtenBuckets
+                                        .computeIfAbsent(t.f0, k -> new HashSet<>())
+                                        .add(Tuple2.of(t.f1, t.f2)));
+
+        snapshotIdentifierToCheck = null;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+        currentWrittenBuckets = new HashSet<>();
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception {
+        SinkRecord record = writeRecord(element);
+
+        BinaryRowData partition = record.partition();
+        int bucket = record.bucket();
+        // partition is a reused BinaryRowData
+        // we first check if the tuple exists to minimize copying
+        if (!currentWrittenBuckets.contains(Tuple2.of(partition, bucket))) {
+            currentWrittenBuckets.add(Tuple2.of(partition.copy(), bucket));
+        }
+    }
+
+    @Override
+    public void snapshotState(StateSnapshotContext context) throws Exception {
+        super.snapshotState(context);
+
+        List<Tuple3<Long, BinaryRowData, Integer>> writtenBucketList = new ArrayList<>();
+        for (Map.Entry<Long, Set<Tuple2<BinaryRowData, Integer>>> entry :
+                writtenBuckets.entrySet()) {
+            for (Tuple2<BinaryRowData, Integer> bucket : entry.getValue()) {
+                writtenBucketList.add(Tuple3.of(entry.getKey(), bucket.f0, bucket.f1));
+            }
+        }
+        writtenBucketState.update(writtenBucketList);
+    }
+
+    @Override
+    protected List<Committable> prepareCommit(boolean blocking, long checkpointId)
+            throws IOException {
+        if (snapshotIdentifierToCheck != null) {
+            Optional<Snapshot> snapshot = findSnapshot(snapshotIdentifierToCheck);
+            if (snapshot.map(s -> s.commitKind() == Snapshot.CommitKind.COMPACT).orElse(false)) {

Review Comment:
   It looks like `AbstractFileStoreWrite` is dealing with the same problem, one is to clear `writer`, one is to clear bucket information.
   
   Can we have a common class to deal with this? Otherwise, this code is a little hard to understand.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org