yunfengzhou-hub commented on code in PR #7027:
URL: https://github.com/apache/paimon/pull/7027#discussion_r2845262054


##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsAction.java:
##########
@@ -50,12 +95,168 @@ public ExpireSnapshotsAction(
         this.olderThan = olderThan;
         this.maxDeletes = maxDeletes;
         this.options = options;
+        this.parallelism = parallelism;
     }
 
+    /** Returns true if forceStartFlinkJob is enabled and parallelism is 
greater than 1. */
+    private boolean isParallelMode() {
+        return forceStartFlinkJob && parallelism != null && parallelism > 1;
+    }
+
+    @Override
+    public void run() throws Exception {
+        if (parallelism != null && parallelism > 1 && !forceStartFlinkJob) {
+            throw new IllegalArgumentException(
+                    "Parallel expire mode requires both --parallelism > 1 and 
--force_start_flink_job enabled.");
+        }
+        if (isParallelMode()) {
+            // Parallel mode: build and execute Flink job (multi parallelism)
+            build();
+            execute(this.getClass().getSimpleName());
+        } else if (forceStartFlinkJob) {
+            // Serial mode but forced to run as Flink job (single parallelism)
+            super.run();
+        } else {
+            // Serial mode: execute locally
+            executeLocally();
+        }
+    }
+
+    @Override
     public void executeLocally() throws Exception {
         ExpireSnapshotsProcedure expireSnapshotsProcedure = new 
ExpireSnapshotsProcedure();
         expireSnapshotsProcedure.withCatalog(catalog);
         expireSnapshotsProcedure.call(
                 null, database + "." + table, retainMax, retainMin, olderThan, 
maxDeletes, options);
     }
+
+    @Override
+    public void build() throws Exception {
+        if (!isParallelMode()) {
+            // Not in parallel mode, nothing to build
+            return;
+        }
+
+        // Prepare table and config using shared method
+        Pair<FileStoreTable, ExpireConfig> prepared =
+                resolveExpireTableAndConfig(
+                        catalog.getTable(Identifier.fromString(database + "." 
+ table)),

Review Comment:
   In line 168 there is
   
   ```java
   Identifier identifier = new Identifier(database, table);
   ```
   
   We can move this line to above to reuse the created identifier variable here.



##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsAction.java:
##########
@@ -50,12 +95,168 @@ public ExpireSnapshotsAction(
         this.olderThan = olderThan;
         this.maxDeletes = maxDeletes;
         this.options = options;
+        this.parallelism = parallelism;
     }
 
+    /** Returns true if forceStartFlinkJob is enabled and parallelism is 
greater than 1. */
+    private boolean isParallelMode() {
+        return forceStartFlinkJob && parallelism != null && parallelism > 1;
+    }
+
+    @Override
+    public void run() throws Exception {

Review Comment:
   In case this method is overridden, this class needs not implement 
`LocalAction` anymore. Still you can keep the `executeLocally` method for 
internal use.



##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsAction.java:
##########
@@ -50,12 +95,168 @@ public ExpireSnapshotsAction(
         this.olderThan = olderThan;
         this.maxDeletes = maxDeletes;
         this.options = options;
+        this.parallelism = parallelism;
     }
 
+    /** Returns true if forceStartFlinkJob is enabled and parallelism is 
greater than 1. */
+    private boolean isParallelMode() {
+        return forceStartFlinkJob && parallelism != null && parallelism > 1;

Review Comment:
   Forcing users to set the parallelism of this action might increase their 
burden to understand the logic of Paimon. It might be better to derive the 
parallelism of the job automatically on its own by default.
   
   I noticed that `parallelism` is mainly used to distribute 
`SnapshotExpireTask`s evenly between the `RangePartitionedExpireFunction` 
subtasks, so this PR created a `List<List<SnapshotExpireTask>>` with 
`parallelism` batches. Compared with this design, a better way for Flink might 
be to have the subtasks directly fetch from the original list, continuing to 
consume the next task after completing the former one. On the one hand, this 
would make it unnecessary to decide the parallelism of the job from the 
beginning. On the other hand, this could also achieve better dynamic 
rebalancing during runtime in case different `SnapshotExpireTask`s have 
different workloads.
   
   Besides, the Flink configuration might be a better place to set the job's 
parallelism, compared with Action arguments.



##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/expire/SnapshotExpireSink.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.paimon.flink.expire;
+
+import org.apache.paimon.annotation.VisibleForTesting;
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.flink.FlinkCatalogFactory;
+import org.apache.paimon.operation.expire.DeletionReport;
+import org.apache.paimon.operation.expire.ExpireSnapshotsExecutor;
+import org.apache.paimon.operation.expire.SnapshotExpireTask;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.table.FileStoreTable;
+
+import org.apache.flink.api.connector.sink2.InitContext;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.api.connector.sink2.WriterInitContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Sink that collects deletion reports from parallel expire workers, 
aggregates the results, and
+ * performs the final commit operations using Sink V2 API.
+ *
+ * <p>In the sink phase (committer), this sink:
+ *
+ * <ul>
+ *   <li>Collects all deletion reports from workers
+ *   <li>Deletes manifest files serially in snapshot ID order (to avoid 
concurrent deletion issues)
+ *   <li>Deletes snapshot metadata files
+ *   <li>Commits changelogs (for changelogDecoupled mode)
+ *   <li>Cleans empty directories
+ *   <li>Updates earliest hint
+ * </ul>
+ */
+public class SnapshotExpireSink implements Sink<DeletionReport> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SnapshotExpireSink.class);
+
+    private final Map<String, String> catalogConfig;
+    private final Identifier identifier;
+    private final long endExclusiveId;
+    private final Set<String> manifestSkippingSet;
+    private final List<SnapshotExpireTask> manifestTasks;
+    private final List<SnapshotExpireTask> snapshotFileTasks;
+
+    public SnapshotExpireSink(
+            Map<String, String> catalogConfig,
+            Identifier identifier,
+            long endExclusiveId,
+            Set<String> manifestSkippingSet,
+            List<SnapshotExpireTask> manifestTasks,
+            List<SnapshotExpireTask> snapshotFileTasks) {
+        this.catalogConfig = catalogConfig;
+        this.identifier = identifier;
+        this.endExclusiveId = endExclusiveId;
+        this.manifestSkippingSet = manifestSkippingSet;
+        this.manifestTasks = manifestTasks;
+        this.snapshotFileTasks = snapshotFileTasks;
+    }
+
+    /**
+     * Do not annotate with <code>@override</code> here to maintain 
compatibility with Flink 2.0+.
+     */
+    public SinkWriter<DeletionReport> createWriter(InitContext context) throws 
IOException {
+        return new ExpireSinkWriter(initExecutor());
+    }
+
+    @Override
+    public SinkWriter<DeletionReport> createWriter(WriterInitContext context) 
throws IOException {
+        return new ExpireSinkWriter(initExecutor());
+    }
+
+    /**
+     * Initializes and returns the executor. Subclasses can override this 
method to provide a custom
+     * executor for testing.
+     */
+    @VisibleForTesting
+    protected ExpireSnapshotsExecutor initExecutor() {
+        try {
+            Options options = Options.fromMap(catalogConfig);
+            Catalog catalog = FlinkCatalogFactory.createPaimonCatalog(options);
+            FileStoreTable table = (FileStoreTable) 
catalog.getTable(identifier);
+            return new ExpireSnapshotsExecutor(
+                    table.snapshotManager(),
+                    table.store().newSnapshotDeletion(),
+                    table.changelogManager());
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to initialize executor", e);
+        }
+    }
+
+    /** SinkWriter that collects reports and performs commit on flush. */
+    private class ExpireSinkWriter implements SinkWriter<DeletionReport> {
+
+        private final Map<BinaryRow, Set<Integer>> globalDeletionBuckets = new 
HashMap<>();
+        private final ExpireSnapshotsExecutor executor;
+
+        ExpireSinkWriter(ExpireSnapshotsExecutor executor) {
+            this.executor = executor;
+        }
+
+        @Override
+        public void write(DeletionReport report, Context context) {
+            if (!report.isSkipped()) {
+                report.deletionBuckets()
+                        .forEach(
+                                (partition, buckets) ->
+                                        globalDeletionBuckets
+                                                .computeIfAbsent(partition, k 
-> new HashSet<>())
+                                                .addAll(buckets));
+            }
+        }
+
+        @Override
+        public void flush(boolean endOfInput) {
+            if (!endOfInput) {
+                return;
+            }
+
+            LOG.info(
+                    "Expire sink received: {} manifest tasks, {} snapshot 
tasks",
+                    manifestTasks.size(),
+                    snapshotFileTasks.size());
+
+            // 1. Clean empty directories
+            executor.cleanEmptyDirectories(globalDeletionBuckets);

Review Comment:
   The implementation here should be aware of the configuration 
`snapshot.clean-empty-directories`.



##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireSnapshotsAction.java:
##########
@@ -50,12 +95,168 @@ public ExpireSnapshotsAction(
         this.olderThan = olderThan;
         this.maxDeletes = maxDeletes;
         this.options = options;
+        this.parallelism = parallelism;
     }
 
+    /** Returns true if forceStartFlinkJob is enabled and parallelism is 
greater than 1. */
+    private boolean isParallelMode() {
+        return forceStartFlinkJob && parallelism != null && parallelism > 1;
+    }
+
+    @Override
+    public void run() throws Exception {
+        if (parallelism != null && parallelism > 1 && !forceStartFlinkJob) {
+            throw new IllegalArgumentException(
+                    "Parallel expire mode requires both --parallelism > 1 and 
--force_start_flink_job enabled.");
+        }
+        if (isParallelMode()) {
+            // Parallel mode: build and execute Flink job (multi parallelism)
+            build();
+            execute(this.getClass().getSimpleName());
+        } else if (forceStartFlinkJob) {
+            // Serial mode but forced to run as Flink job (single parallelism)
+            super.run();
+        } else {
+            // Serial mode: execute locally
+            executeLocally();
+        }
+    }
+
+    @Override
     public void executeLocally() throws Exception {
         ExpireSnapshotsProcedure expireSnapshotsProcedure = new 
ExpireSnapshotsProcedure();
         expireSnapshotsProcedure.withCatalog(catalog);
         expireSnapshotsProcedure.call(
                 null, database + "." + table, retainMax, retainMin, olderThan, 
maxDeletes, options);
     }
+
+    @Override
+    public void build() throws Exception {
+        if (!isParallelMode()) {
+            // Not in parallel mode, nothing to build
+            return;
+        }
+
+        // Prepare table and config using shared method
+        Pair<FileStoreTable, ExpireConfig> prepared =
+                resolveExpireTableAndConfig(

Review Comment:
   The introduction of method `resolveExpireTableAndConfig` seems to increase 
the code complexity a little bit. It might be better to directly put its 
implementation here.



##########
paimon-core/src/main/java/org/apache/paimon/operation/expire/ExpireSnapshotsPlan.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.paimon.operation.expire;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The plan for snapshot expiration, containing four groups of tasks organized 
by deletion phase.
+ *
+ * <p>The plan organizes tasks into four groups for correct deletion order:
+ *
+ * <ul>
+ *   <li>{@link #dataFileTasks()}: Phase 1a - Delete data files (can be 
parallelized)
+ *   <li>{@link #changelogFileTasks()}: Phase 1b - Delete changelog files (can 
be parallelized)
+ *   <li>{@link #manifestTasks()}: Phase 2a - Delete manifest files (serial)
+ *   <li>{@link #snapshotFileTasks()}: Phase 2b - Delete snapshot metadata 
files (serial)
+ * </ul>
+ */
+public class ExpireSnapshotsPlan implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final ExpireSnapshotsPlan EMPTY =
+            new ExpireSnapshotsPlan(
+                    Collections.emptyList(),
+                    Collections.emptyList(),
+                    Collections.emptyList(),
+                    Collections.emptyList(),
+                    null,
+                    0,
+                    0);
+
+    private final List<SnapshotExpireTask> dataFileTasks;
+    private final List<SnapshotExpireTask> changelogFileTasks;
+    private final List<SnapshotExpireTask> manifestTasks;
+    private final List<SnapshotExpireTask> snapshotFileTasks;
+    private final ProtectionSet protectionSet;
+    private final long beginInclusiveId;
+    private final long endExclusiveId;
+
+    public ExpireSnapshotsPlan(
+            List<SnapshotExpireTask> dataFileTasks,
+            List<SnapshotExpireTask> changelogFileTasks,
+            List<SnapshotExpireTask> manifestTasks,
+            List<SnapshotExpireTask> snapshotFileTasks,
+            ProtectionSet protectionSet,
+            long beginInclusiveId,
+            long endExclusiveId) {
+        this.dataFileTasks = dataFileTasks;
+        this.changelogFileTasks = changelogFileTasks;
+        this.manifestTasks = manifestTasks;
+        this.snapshotFileTasks = snapshotFileTasks;
+        this.protectionSet = protectionSet;
+        this.beginInclusiveId = beginInclusiveId;
+        this.endExclusiveId = endExclusiveId;
+    }
+
+    public static ExpireSnapshotsPlan empty() {
+        return EMPTY;
+    }
+
+    public boolean isEmpty() {
+        return dataFileTasks.isEmpty()
+                && changelogFileTasks.isEmpty()
+                && manifestTasks.isEmpty()
+                && snapshotFileTasks.isEmpty();
+    }
+
+    /** Get data file deletion tasks (Phase 1a). These can be executed in 
parallel. */
+    public List<SnapshotExpireTask> dataFileTasks() {
+        return dataFileTasks;
+    }
+
+    /** Get changelog file deletion tasks (Phase 1b). These can be executed in 
parallel. */
+    public List<SnapshotExpireTask> changelogFileTasks() {
+        return changelogFileTasks;
+    }
+
+    /** Get manifest deletion tasks (Phase 2a). These should be executed 
serially. */
+    public List<SnapshotExpireTask> manifestTasks() {
+        return manifestTasks;
+    }
+
+    /** Get snapshot file deletion tasks (Phase 2b). These should be executed 
serially. */
+    public List<SnapshotExpireTask> snapshotFileTasks() {
+        return snapshotFileTasks;
+    }
+
+    /**
+     * Partition tasks into groups by snapshot ID range for parallel mode 
execution.
+     *
+     * <p>Each group contains tasks for a contiguous range of snapshots, with 
dataFileTasks first,
+     * then changelogFileTasks. This ensures:
+     *
+     * <ul>
+     *   <li>Same snapshot range tasks are processed by the same worker
+     *   <li>Within each worker, data files are deleted before changelog files
+     * </ul>
+     *
+     * <p>For example, with beginInclusiveId=1, endExclusiveId=11, 
parallelism=3:
+     *
+     * <ul>
+     *   <li>Worker 0 (snapshot 1-4): [dataTask(2,3,4), changelogTask(1,2,3,4)]
+     *   <li>Worker 1 (snapshot 5-8): [dataTask(5,6,7,8), 
changelogTask(5,6,7,8)]
+     *   <li>Worker 2 (snapshot 9-11): [dataTask(9,10,11), changelogTask(9,10)]
+     * </ul>
+     *
+     * @param parallelism target parallelism for distribution
+     * @return list of task groups, one per worker
+     */
+    public List<List<SnapshotExpireTask>> partitionTasksBySnapshotRange(int 
parallelism) {

Review Comment:
   It seems that this PR divides the expire process of a snapshot into 
different `SnapshotExpireTask`s first, and then sends the tasks of the same 
snapshot to the same Flink subtask. If this is the case, why should this PR 
make the division first? Would the following implementation simpler?
   
   ```java
   // calculates the start and end snapshot id before this part of code
   
   env.fromSequence(startId, endId)
       .flatMap(new SnapshotExpireFunction()) // deletes the data, manifest, 
and snapshot file sequentially
       .sinkTo(new SnapshotExpireSink());
   ```



##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/expire/SnapshotExpireSink.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.paimon.flink.expire;
+
+import org.apache.paimon.annotation.VisibleForTesting;
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.flink.FlinkCatalogFactory;
+import org.apache.paimon.operation.expire.DeletionReport;
+import org.apache.paimon.operation.expire.ExpireSnapshotsExecutor;
+import org.apache.paimon.operation.expire.SnapshotExpireTask;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.table.FileStoreTable;
+
+import org.apache.flink.api.connector.sink2.InitContext;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.api.connector.sink2.WriterInitContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Sink that collects deletion reports from parallel expire workers, 
aggregates the results, and
+ * performs the final commit operations using Sink V2 API.
+ *
+ * <p>In the sink phase (committer), this sink:
+ *
+ * <ul>
+ *   <li>Collects all deletion reports from workers
+ *   <li>Deletes manifest files serially in snapshot ID order (to avoid 
concurrent deletion issues)
+ *   <li>Deletes snapshot metadata files
+ *   <li>Commits changelogs (for changelogDecoupled mode)
+ *   <li>Cleans empty directories
+ *   <li>Updates earliest hint
+ * </ul>
+ */
+public class SnapshotExpireSink implements Sink<DeletionReport> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SnapshotExpireSink.class);
+
+    private final Map<String, String> catalogConfig;
+    private final Identifier identifier;
+    private final long endExclusiveId;
+    private final Set<String> manifestSkippingSet;
+    private final List<SnapshotExpireTask> manifestTasks;
+    private final List<SnapshotExpireTask> snapshotFileTasks;
+
+    public SnapshotExpireSink(
+            Map<String, String> catalogConfig,
+            Identifier identifier,
+            long endExclusiveId,
+            Set<String> manifestSkippingSet,
+            List<SnapshotExpireTask> manifestTasks,
+            List<SnapshotExpireTask> snapshotFileTasks) {
+        this.catalogConfig = catalogConfig;
+        this.identifier = identifier;
+        this.endExclusiveId = endExclusiveId;
+        this.manifestSkippingSet = manifestSkippingSet;
+        this.manifestTasks = manifestTasks;
+        this.snapshotFileTasks = snapshotFileTasks;
+    }
+
+    /**
+     * Do not annotate with <code>@override</code> here to maintain 
compatibility with Flink 2.0+.
+     */
+    public SinkWriter<DeletionReport> createWriter(InitContext context) throws 
IOException {
+        return new ExpireSinkWriter(initExecutor());
+    }
+
+    @Override

Review Comment:
   Do we need the following javadoc here?
   ```java
   /**
    * Do not annotate with <code>@override</code> here to maintain 
compatibility with Flink 1.18-.
    */
   ```



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