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


##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreSinkCompactor.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.connector.StatefulPrecommittingSinkWriter;
+import org.apache.flink.table.store.file.FileStore;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.operation.FileStoreScan;
+import org.apache.flink.table.store.file.predicate.PredicateConverter;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.store.file.writer.RecordWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/** A dedicated {@link SinkWriter} for manual triggered compaction. */
+public class StoreSinkCompactor implements 
StatefulPrecommittingSinkWriter<Void> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(StoreSinkCompactor.class);
+
+    private final int subTaskId;
+    private final int numOfParallelInstances;
+
+    private final FileStore fileStore;
+    private final Map<String, String> partitionSpec;
+    private final Map<BinaryRowData, Map<Integer, RecordWriter>> writers;

Review Comment:
   We don't need to store writers now.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/CompactWriter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.writer;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.ValueKind;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.data.DataFileWriter;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.mergetree.compact.CompactManager;
+import org.apache.flink.table.store.file.mergetree.compact.CompactUnit;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A {@link RecordWriter} implementation that only perform compaction on 
existing records and does
+ * not generate new records.
+ */
+public class CompactWriter implements RecordWriter {
+
+    private final CompactUnit unit;
+    private final ExecutorService compactExecutor;
+    private final CompactManager compactor;
+    private final DataFileWriter dataFileWriter;
+
+    public CompactWriter(
+            CompactUnit unit,
+            Comparator<RowData> keyComparator,
+            long targetFileSize,
+            CompactManager.Rewriter rewriter,
+            DataFileWriter dataFileWriter) {
+        this.unit = unit;
+        this.compactExecutor =
+                Executors.newSingleThreadScheduledExecutor(
+                        new ExecutorThreadFactory("compaction-thread"));
+        this.compactor =
+                new CompactManager(
+                        compactExecutor,
+                        (numLevels, runs) ->
+                                
Optional.of(CompactUnit.fromLevelRuns(numLevels - 1, runs)),
+                        keyComparator,
+                        targetFileSize,
+                        rewriter);
+        this.dataFileWriter = dataFileWriter;
+    }
+
+    @Override
+    public Increment prepareCommit() throws IOException, InterruptedException {
+        List<DataFileMeta> compactBefore = new ArrayList<>();
+        List<DataFileMeta> compactAfter = new ArrayList<>();
+        if (compactor.isCompactionFinished()) {
+            compactor.submitCompaction(unit, true);
+        }
+        while (!compactor.isCompactionFinished()) {
+            try {
+                compactor
+                        .finishCompaction(true)
+                        .ifPresent(
+                                result -> {
+                                    compactBefore.addAll(result.before());
+                                    compactAfter.addAll(result.after());
+                                });
+            } catch (ExecutionException e) {
+                throw new IOException(e.getCause());
+            }
+        }
+        return Increment.forCompact(compactBefore, compactAfter);
+    }
+
+    @Override
+    public List<DataFileMeta> close() throws Exception {
+        compactExecutor.shutdownNow();
+        List<DataFileMeta> delete = new ArrayList<>();
+        try {
+            compactor.finishCompaction(true).ifPresent(result -> 
delete.addAll(result.after()));

Review Comment:
   Do not block `close` here.
   Currently we don't have a good way to remove compaction's temporary files. 
This requires `CompactManager` to keep track of the intermediate files, and you 
can just leave a TODO here for now.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/CompactWriter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.writer;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.ValueKind;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.data.DataFileWriter;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.mergetree.compact.CompactManager;
+import org.apache.flink.table.store.file.mergetree.compact.CompactUnit;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A {@link RecordWriter} implementation that only perform compaction on 
existing records and does
+ * not generate new records.
+ */
+public class CompactWriter implements RecordWriter {
+
+    private final CompactUnit unit;
+    private final ExecutorService compactExecutor;
+    private final CompactManager compactor;
+    private final DataFileWriter dataFileWriter;
+
+    public CompactWriter(
+            CompactUnit unit,
+            Comparator<RowData> keyComparator,
+            long targetFileSize,
+            CompactManager.Rewriter rewriter,
+            DataFileWriter dataFileWriter) {
+        this.unit = unit;
+        this.compactExecutor =
+                Executors.newSingleThreadScheduledExecutor(
+                        new ExecutorThreadFactory("compaction-thread"));
+        this.compactor =
+                new CompactManager(
+                        compactExecutor,
+                        (numLevels, runs) ->
+                                
Optional.of(CompactUnit.fromLevelRuns(numLevels - 1, runs)),
+                        keyComparator,
+                        targetFileSize,
+                        rewriter);
+        this.dataFileWriter = dataFileWriter;
+    }
+
+    @Override
+    public Increment prepareCommit() throws IOException, InterruptedException {
+        List<DataFileMeta> compactBefore = new ArrayList<>();
+        List<DataFileMeta> compactAfter = new ArrayList<>();
+        if (compactor.isCompactionFinished()) {
+            compactor.submitCompaction(unit, true);
+        }
+        while (!compactor.isCompactionFinished()) {

Review Comment:
   Remove `while`? There is only one compaction.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/CompactWriter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.writer;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.ValueKind;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.data.DataFileWriter;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.mergetree.compact.CompactManager;
+import org.apache.flink.table.store.file.mergetree.compact.CompactUnit;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A {@link RecordWriter} implementation that only perform compaction on 
existing records and does
+ * not generate new records.
+ */
+public class CompactWriter implements RecordWriter {
+
+    private final CompactUnit unit;
+    private final ExecutorService compactExecutor;
+    private final CompactManager compactor;
+    private final DataFileWriter dataFileWriter;
+
+    public CompactWriter(
+            CompactUnit unit,
+            Comparator<RowData> keyComparator,
+            long targetFileSize,
+            CompactManager.Rewriter rewriter,
+            DataFileWriter dataFileWriter) {
+        this.unit = unit;
+        this.compactExecutor =
+                Executors.newSingleThreadScheduledExecutor(
+                        new ExecutorThreadFactory("compaction-thread"));
+        this.compactor =
+                new CompactManager(
+                        compactExecutor,
+                        (numLevels, runs) ->
+                                
Optional.of(CompactUnit.fromLevelRuns(numLevels - 1, runs)),
+                        keyComparator,
+                        targetFileSize,
+                        rewriter);
+        this.dataFileWriter = dataFileWriter;
+    }
+
+    @Override
+    public Increment prepareCommit() throws IOException, InterruptedException {
+        List<DataFileMeta> compactBefore = new ArrayList<>();
+        List<DataFileMeta> compactAfter = new ArrayList<>();
+        if (compactor.isCompactionFinished()) {

Review Comment:
   If compaction not finish, just do nothing?
   I think here we can just check `CompactionFinished`.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/CompactWriter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.writer;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.ValueKind;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.data.DataFileWriter;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.mergetree.compact.CompactManager;
+import org.apache.flink.table.store.file.mergetree.compact.CompactUnit;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A {@link RecordWriter} implementation that only perform compaction on 
existing records and does
+ * not generate new records.
+ */
+public class CompactWriter implements RecordWriter {
+
+    private final CompactUnit unit;
+    private final ExecutorService compactExecutor;
+    private final CompactManager compactor;
+    private final DataFileWriter dataFileWriter;
+
+    public CompactWriter(
+            CompactUnit unit,
+            Comparator<RowData> keyComparator,
+            long targetFileSize,
+            CompactManager.Rewriter rewriter,

Review Comment:
   Can we just pass a compact manager here?
   And we can not new a separate thread for this bucket.



-- 
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