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


##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/TableStoreManagedFactory.java:
##########
@@ -183,6 +204,84 @@ public void onDropTable(Context context, boolean 
ignoreIfNotExists) {
     @Override
     public Map<String, String> onCompactTable(
             Context context, CatalogPartitionSpec catalogPartitionSpec) {
-        throw new UnsupportedOperationException("Not implement yet");
+        Map<String, String> newOptions = new 
HashMap<>(context.getCatalogTable().getOptions());
+        FileStore fileStore = buildTableStore(context).buildFileStore();
+        FileStoreScan.Plan plan =
+                fileStore
+                        .newScan()
+                        .withPartitionFilter(
+                                PredicateConverter.CONVERTER.fromMap(
+                                        
catalogPartitionSpec.getPartitionSpec(),
+                                        fileStore.partitionType()))
+                        .plan();
+
+        Preconditions.checkState(
+                plan.snapshotId() != null && !plan.files().isEmpty(),
+                "The specified %s to compact does not exist any snapshot",
+                catalogPartitionSpec.getPartitionSpec().isEmpty()
+                        ? "table"
+                        : String.format("partition %s", 
catalogPartitionSpec.getPartitionSpec()));
+        Map<BinaryRowData, Map<Integer, List<DataFileMeta>>> groupBy = 
plan.groupByPartFiles();
+        if 
(!Boolean.parseBoolean(newOptions.get(COMPACTION_RESCALE_BUCKET.key()))) {
+            groupBy =
+                    pickManifest(
+                            groupBy,
+                            new 
FileStoreOptions(Configuration.fromMap(newOptions))
+                                    .mergeTreeOptions(),
+                            new 
KeyComparatorSupplier(fileStore.partitionType()).get());
+        }
+        try {
+            newOptions.put(
+                    COMPACTION_SCANNED_MANIFEST.key(),
+                    Base64.getEncoder()
+                            .encodeToString(
+                                    InstantiationUtil.serializeObject(
+                                            new PartitionedManifestMeta(
+                                                    plan.snapshotId(), 
groupBy))));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return newOptions;
+    }
+
+    @VisibleForTesting
+    Map<BinaryRowData, Map<Integer, List<DataFileMeta>>> pickManifest(

Review Comment:
   > You have picked files here, but how to make sure that writer will compact 
these files?
   
   As offline discussed, the main purpose for `ALTER TABLE COMPACT` is to 
squeeze those files which have key range overlapped or too small. It is not 
exactly what universal compaction does. As a result, when after picking these 
files at the planning phase, the runtime should not pick them again, because 
they are already picked. So `FileStoreWriteImpl` should create different 
compact strategies for ① the auto-compaction triggered by ordinary writes v.s. 
② the manual triggered compaction. For the latter, the strategy should directly 
return all the files it receives.



-- 
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-unsubscr...@flink.apache.org

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

Reply via email to