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/06/16 11:47:31 UTC

[GitHub] [flink-table-store] JingsongLi opened a new pull request, #160: [FLINK-28098] Refactor table store compactor

JingsongLi opened a new pull request, #160:
URL: https://github.com/apache/flink-table-store/pull/160

   There is currently some room for code optimization on the path of the compact.
   
   - It uses the CompactManager, which actually only uses the CompactTask.
   - It also doesn't use an asynchronous thread to complete the compaction.
   - There should be FileStoreTable to provide unified table-level compaction.


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


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #160: [FLINK-28098] Refactor table store compactor

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #160:
URL: https://github.com/apache/flink-table-store/pull/160#discussion_r899682545


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWrite.java:
##########
@@ -40,10 +44,12 @@ RecordWriter<T> createWriter(
     RecordWriter<T> createEmptyWriter(
             BinaryRowData partition, int bucket, ExecutorService compactExecutor);
 
-    /** Create a compact {@link RecordWriter} from partition, bucket and restore files. */
-    RecordWriter<T> createCompactWriter(
-            BinaryRowData partition,
-            int bucket,
-            ExecutorService compactExecutor,
-            List<DataFileMeta> restoredFiles);
+    /**
+     * Create a {@link Callable} compactor from partition, bucket.
+     *
+     * @param compactFiles input files of compaction. When it is null, will automatically read all
+     *     files of the current bucket.
+     */
+    Callable<CompactResult> createCompactWriter(
+            BinaryRowData partition, int bucket, @Nullable List<DataFileMeta> compactFiles);

Review Comment:
   Actually, here we can just pass a null, but for the performance, we reuse compact files.



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


[GitHub] [flink-table-store] LadyForest commented on a diff in pull request #160: [FLINK-28098] Refactor table store compactor

Posted by GitBox <gi...@apache.org>.
LadyForest commented on code in PR #160:
URL: https://github.com/apache/flink-table-store/pull/160#discussion_r899068242


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/AppendOnlyFileStoreWrite.java:
##########
@@ -71,11 +75,8 @@ public RecordWriter<RowData> createEmptyWriter(
     }
 
     @Override
-    public RecordWriter<RowData> createCompactWriter(
-            BinaryRowData partition,
-            int bucket,
-            ExecutorService compactExecutor,
-            List<DataFileMeta> restoredFiles) {
+    public Callable<CompactResult> createCompactWriter(
+            BinaryRowData partition, int bucket, @Nullable List<DataFileMeta> compactFiles) {

Review Comment:
   Just curious about which condition `compactFiles` is passed as `null`



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


[GitHub] [flink-table-store] LadyForest commented on a diff in pull request #160: [FLINK-28098] Refactor table store compactor

Posted by GitBox <gi...@apache.org>.
LadyForest commented on code in PR #160:
URL: https://github.com/apache/flink-table-store/pull/160#discussion_r899069363


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWrite.java:
##########
@@ -40,10 +44,12 @@ RecordWriter<T> createWriter(
     RecordWriter<T> createEmptyWriter(
             BinaryRowData partition, int bucket, ExecutorService compactExecutor);
 
-    /** Create a compact {@link RecordWriter} from partition, bucket and restore files. */
-    RecordWriter<T> createCompactWriter(
-            BinaryRowData partition,
-            int bucket,
-            ExecutorService compactExecutor,
-            List<DataFileMeta> restoredFiles);
+    /**
+     * Create a {@link Callable} compactor from partition, bucket.
+     *
+     * @param compactFiles input files of compaction. When it is null, will automatically read all
+     *     files of the current bucket.
+     */
+    Callable<CompactResult> createCompactWriter(
+            BinaryRowData partition, int bucket, @Nullable List<DataFileMeta> compactFiles);

Review Comment:
   Just curious about which condition `compactFiles` is passed as `null`



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


[GitHub] [flink-table-store] LadyForest commented on a diff in pull request #160: [FLINK-28098] Refactor table store compactor

Posted by GitBox <gi...@apache.org>.
LadyForest commented on code in PR #160:
URL: https://github.com/apache/flink-table-store/pull/160#discussion_r899064529


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableCompact.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.table.sink;
+
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.mergetree.compact.CompactResult;
+import org.apache.flink.table.store.file.operation.FileStoreScan;
+import org.apache.flink.table.store.file.operation.FileStoreWrite;
+import org.apache.flink.table.store.file.predicate.PredicateConverter;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+
+/** An abstraction layer above {@link FileStoreWrite#createCompactWriter} to provide compaction. */
+public class TableCompact {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TableCompact.class);
+
+    private final FileStoreScan scan;
+    private final FileStoreWrite<?> write;
+    private final RowType partitionType;
+
+    private BiPredicate<BinaryRowData, Integer> partBucketFilter;
+
+    public TableCompact(FileStoreScan scan, FileStoreWrite<?> write, RowType partitionType) {
+        this.scan = scan;
+        this.write = write;
+        this.partitionType = partitionType;
+    }
+
+    public TableCompact withPartitions(Map<String, String> partitionSpec) {

Review Comment:
   ```suggestion
       public TableCompact withPartition(Map<String, String> partitionSpec) {
   ```
   Nit: actually there's just one partition can be specified



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


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #160: [FLINK-28098] Refactor table store compactor

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #160:
URL: https://github.com/apache/flink-table-store/pull/160#discussion_r899682545


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWrite.java:
##########
@@ -40,10 +44,12 @@ RecordWriter<T> createWriter(
     RecordWriter<T> createEmptyWriter(
             BinaryRowData partition, int bucket, ExecutorService compactExecutor);
 
-    /** Create a compact {@link RecordWriter} from partition, bucket and restore files. */
-    RecordWriter<T> createCompactWriter(
-            BinaryRowData partition,
-            int bucket,
-            ExecutorService compactExecutor,
-            List<DataFileMeta> restoredFiles);
+    /**
+     * Create a {@link Callable} compactor from partition, bucket.
+     *
+     * @param compactFiles input files of compaction. When it is null, will automatically read all
+     *     files of the current bucket.
+     */
+    Callable<CompactResult> createCompactWriter(
+            BinaryRowData partition, int bucket, @Nullable List<DataFileMeta> compactFiles);

Review Comment:
   Actually, here we can just pass a null, (bucket know its all files) but for the performance, we reuse compact files.



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


[GitHub] [flink-table-store] JingsongLi merged pull request #160: [FLINK-28098] Refactor table store compactor

Posted by GitBox <gi...@apache.org>.
JingsongLi merged PR #160:
URL: https://github.com/apache/flink-table-store/pull/160


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


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #160: [FLINK-28098] Refactor table store compactor

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #160:
URL: https://github.com/apache/flink-table-store/pull/160#discussion_r899682102


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/table/sink/TableCompact.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.table.sink;
+
+import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.mergetree.compact.CompactResult;
+import org.apache.flink.table.store.file.operation.FileStoreScan;
+import org.apache.flink.table.store.file.operation.FileStoreWrite;
+import org.apache.flink.table.store.file.predicate.PredicateConverter;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+
+/** An abstraction layer above {@link FileStoreWrite#createCompactWriter} to provide compaction. */
+public class TableCompact {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TableCompact.class);
+
+    private final FileStoreScan scan;
+    private final FileStoreWrite<?> write;
+    private final RowType partitionType;
+
+    private BiPredicate<BinaryRowData, Integer> partBucketFilter;
+
+    public TableCompact(FileStoreScan scan, FileStoreWrite<?> write, RowType partitionType) {
+        this.scan = scan;
+        this.write = write;
+        this.partitionType = partitionType;
+    }
+
+    public TableCompact withPartitions(Map<String, String> partitionSpec) {

Review Comment:
   `partitionSpec` can be partial, which means there can be many partitions.



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