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


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/FullChangelogMergeTreeCompactRewriter.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.file.mergetree.compact;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.KeyValue;
+import org.apache.flink.table.store.file.compact.CompactResult;
+import org.apache.flink.table.store.file.io.DataFileMeta;
+import org.apache.flink.table.store.file.io.KeyValueFileReaderFactory;
+import org.apache.flink.table.store.file.io.KeyValueFileWriterFactory;
+import org.apache.flink.table.store.file.io.RollingFileWriter;
+import org.apache.flink.table.store.file.mergetree.MergeTreeReaders;
+import org.apache.flink.table.store.file.mergetree.SortedRun;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.file.utils.RecordReaderIterator;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+/** A {@link MergeTreeCompactRewriter} which produces changelog files for each 
full compaction. */
+public class FullChangelogMergeTreeCompactRewriter extends 
MergeTreeCompactRewriter {
+
+    private final int maxLevel;
+
+    public FullChangelogMergeTreeCompactRewriter(
+            int maxLevel,
+            KeyValueFileReaderFactory readerFactory,
+            KeyValueFileWriterFactory writerFactory,
+            Comparator<RowData> keyComparator,
+            MergeFunction<KeyValue> mergeFunction) {
+        super(readerFactory, writerFactory, keyComparator, mergeFunction);
+        this.maxLevel = maxLevel;
+    }
+
+    @Override
+    public void rewrite(
+            int outputLevel,
+            boolean dropDelete,
+            List<List<SortedRun>> sections,
+            CompactResult toUpdate)
+            throws Exception {
+        addBefore(sections, toUpdate);
+
+        if (outputLevel == maxLevel) {
+            Preconditions.checkArgument(
+                    dropDelete,
+                    "Delete records should be dropped from result of full 
compaction. This is unexpected.");
+            rewriteFullCompaction(sections, toUpdate);
+        } else {
+            rewriteCompaction(outputLevel, dropDelete, sections, toUpdate);
+        }
+    }
+
+    private void rewriteFullCompaction(List<List<SortedRun>> sections, 
CompactResult toUpdate)
+            throws Exception {
+        
List<ConcatRecordReader.ReaderSupplier<FullChangelogMergeFunctionWrapper.Result>>
+                sectionReaders = new ArrayList<>();
+        for (List<SortedRun> section : sections) {
+            sectionReaders.add(
+                    () -> {
+                        List<RecordReader<KeyValue>> runReaders = new 
ArrayList<>();
+                        for (SortedRun run : section) {
+                            runReaders.add(MergeTreeReaders.readerForRun(run, 
readerFactory));
+                        }
+                        return new SortMergeReader<>(
+                                runReaders,
+                                keyComparator,
+                                new FullChangelogMergeFunctionWrapper(
+                                        mergeFunction.copy(), maxLevel));
+                    });
+        }
+        RecordReader<FullChangelogMergeFunctionWrapper.Result> sectionsReader =

Review Comment:
   Use `try` like this to ensure closed, maybe another writer will be forgot to 
close?
   ```
           try (
                   RecordReader<FullChangelogMergeFunctionWrapper.Result> 
sectionsReader =
                           ConcatRecordReader.create(sectionReaders);
                   RollingFileWriter<KeyValue, DataFileMeta> compactFileWriter =
                           
writerFactory.createRollingMergeTreeFileWriter(maxLevel);
                   RollingFileWriter<KeyValue, DataFileMeta> 
changelogFileWriter =
                           
writerFactory.createRollingChangelogFileWriter(maxLevel)
                   ) {
               RecordReaderIterator<FullChangelogMergeFunctionWrapper.Result> 
iterator =
                       new RecordReaderIterator<>(sectionsReader);
               while (iterator.hasNext()) {
                   FullChangelogMergeFunctionWrapper.Result result = 
iterator.next();
                   if (result.result() != null) {
                       compactFileWriter.write(result.result());
                   }
                   if (result.before() != null) {
                       changelogFileWriter.write(result.before());
                   }
                   if (result.after() != null) {
                       changelogFileWriter.write(result.after());
                   }
               }
               toUpdate.addAfter(compactFileWriter.result());
               toUpdate.addChangelog(changelogFileWriter.result());
           }
   ```



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/compact/CompactRewriter.java:
##########
@@ -18,15 +18,21 @@
 
 package org.apache.flink.table.store.file.mergetree.compact;
 
+import org.apache.flink.table.store.file.compact.CompactResult;
 import org.apache.flink.table.store.file.io.DataFileMeta;
 import org.apache.flink.table.store.file.mergetree.SortedRun;
 
 import java.util.List;
 
 /** Rewrite sections to the files. */
-@FunctionalInterface
 public interface CompactRewriter {
 
-    List<DataFileMeta> rewrite(int outputLevel, boolean dropDelete, 
List<List<SortedRun>> sections)
+    void rewrite(

Review Comment:
   Can we return a `CompactResult`? And provide a `merge(CompactResult other)` 
to `CompactResult`?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to