JingsongLi commented on code in PR #7313:
URL: https://github.com/apache/paimon/pull/7313#discussion_r3381883728


##########
paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactChainTableProcedure.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.spark.procedure;
+
+import org.apache.paimon.partition.PartitionPredicate;
+import org.apache.paimon.spark.commands.PaimonSparkWriter;
+import org.apache.paimon.spark.util.ScanPlanHelper$;
+import org.apache.paimon.spark.utils.SparkProcedureUtils;
+import org.apache.paimon.table.ChainGroupReadTable;
+import org.apache.paimon.table.FallbackReadFileStoreTable;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.source.Split;
+import org.apache.paimon.utils.ParameterUtils;
+import org.apache.paimon.utils.StringUtils;
+
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.PaimonUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.paimon.utils.Preconditions.checkArgument;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Procedure to compact chain table. Usage:
+ *
+ * <pre><code>
+ *  -- Compact chain table, overwrite default is false
+ *  CALL sys.compact_chain_table(table => 'db.table', partition => 
'dt="20250810",hour="22"', [overwrite => true])
+ * </code></pre>
+ */
+public class CompactChainTableProcedure extends BaseProcedure {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(CompactChainTableProcedure.class);
+
+    private static final ProcedureParameter[] PARAMETERS =
+            new ProcedureParameter[] {
+                ProcedureParameter.required("table", StringType),
+                ProcedureParameter.required("partition", StringType),
+                ProcedureParameter.optional("overwrite", BooleanType)
+            };
+
+    private static final StructType OUTPUT_TYPE =
+            new StructType(
+                    new StructField[] {
+                        new StructField("result", BooleanType, false, 
Metadata.empty())
+                    });
+
+    protected CompactChainTableProcedure(TableCatalog tableCatalog) {
+        super(tableCatalog);
+    }
+
+    @Override
+    public ProcedureParameter[] parameters() {
+        return PARAMETERS;
+    }
+
+    @Override
+    public StructType outputType() {
+        return OUTPUT_TYPE;
+    }
+
+    @Override
+    public InternalRow[] call(InternalRow args) {
+        Identifier tableIdent = toIdentifier(args.getString(0), 
PARAMETERS[0].name());
+        String partitionStr = args.getString(1);
+        boolean overwrite = !args.isNullAt(2) && args.getBoolean(2);
+        checkArgument(StringUtils.isNotEmpty(partitionStr), "Partition string 
cannot be empty");
+
+        return modifyPaimonTable(
+                tableIdent,
+                t -> {
+                    checkArgument(
+                            t instanceof FallbackReadFileStoreTable,
+                            "Table %s is not a chain table",
+                            tableIdent);
+                    FallbackReadFileStoreTable table = 
(FallbackReadFileStoreTable) t;
+                    checkArgument(
+                            table.other() instanceof ChainGroupReadTable,
+                            "Table %s is not a chain table",
+                            tableIdent);
+                    DataSourceV2Relation relation = createRelation(tableIdent);
+                    boolean success =
+                            execute(
+                                    (ChainGroupReadTable) table.other(),
+                                    relation,
+                                    partitionStr,
+                                    overwrite);
+                    return new InternalRow[] {newInternalRow(success)};
+                });
+    }
+
+    private boolean execute(
+            ChainGroupReadTable table,
+            DataSourceV2Relation relation,
+            String partitionStr,
+            boolean overwrite) {
+        String partition = SparkProcedureUtils.toWhere(partitionStr);
+        FileStoreTable snapshotTable = table.wrapped();
+
+        FallbackReadFileStoreTable.FallbackReadScan scan =
+                (FallbackReadFileStoreTable.FallbackReadScan) table.newScan();
+        PartitionPredicate partitionPredicate =
+                SparkProcedureUtils.convertToPartitionPredicate(
+                        partition, table.schema().logicalPartitionType(), 
spark(), relation);
+
+        // Check if target partition already exists in snapshot branch
+        boolean partitionExists = checkPartitionExists(snapshotTable, 
partition, relation);
+        if (partitionExists) {
+            if (overwrite) {
+                scan.withPartitionFilter(

Review Comment:
   This overwrite path appears to read too much data. When the target partition 
already exists, the snapshot-side predicate is changed to `NOT (target 
partition)`, while the delta side still uses the target partition. However, 
`ChainGroupReadTable.plan()` adds all `mainScan.plan()` splits directly before 
planning the delta/anchor splits, and this procedure later rewrites every 
output row's partition columns to the target partition. That means unrelated 
snapshot partitions can be copied into the compacted target partition during 
overwrite. The overwrite path should only read the chain-merge result needed 
for the target partition, not every snapshot partition except the target.



##########
docs/docs/primary-key-table/chain-table.md:
##########
@@ -191,6 +190,32 @@ you will get the following result:
 +---+----+-----+ 
 ```
 
+- Chain Table Compaction: Merge data from snapshot and delta branches into the 
snapshot branch.
+  This is useful for periodically compacting incremental data into full 
snapshots.
+  You can use the `compact_chain_table` procedure to merge a specific 
partition:
+
+```sql
+CALL sys.compact_chain_table(table => 'default.t', partition => 
'date="20250811"');
+```
+
+After compaction, the data in the snapshot branch will contain the merged 
result from both snapshot
+and delta branches, and subsequent queries will benefit from direct snapshot 
access without
+merge-on-read overhead.
+
+```sql
+select t1, t2, t3 from `default`.`t$branch_snapshot` where date = '20250811';
+```
+
+you will get the following result:
+```text
++---+----+-----+ 

Review Comment:
   Please remove the trailing whitespace in this added result block. `git diff 
--check origin/master...HEAD` reports trailing whitespace on this block, which 
will fail whitespace/style checks.



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