This is an automated email from the ASF dual-hosted git repository.

yuzelin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new 0d6db479f4 [flink] Flink batch job support specify partition with 
max_pt() and max_two_pt()
0d6db479f4 is described below

commit 0d6db479f4879599b2da4fa8cbfad413276720bc
Author: yuzelin <[email protected]>
AuthorDate: Tue Dec 2 22:14:24 2025 +0800

    [flink] Flink batch job support specify partition with max_pt() and 
max_two_pt()
---
 .../flink/lookup/DynamicPartitionLevelLoader.java  | 13 ++--
 .../flink/lookup/DynamicPartitionLoader.java       |  6 +-
 .../flink/lookup/DynamicPartitionNumberLoader.java |  8 +--
 .../flink/lookup/FileStoreLookupFunction.java      |  4 +-
 .../paimon/flink/lookup/PartitionLoader.java       | 14 ++---
 .../paimon/flink/lookup/StaticPartitionLoader.java | 13 ++--
 .../paimon/flink/source/FlinkTableSource.java      | 70 ++++++++++++----------
 .../apache/paimon/flink/BatchFileStoreITCase.java  | 40 +++++++++++++
 8 files changed, 108 insertions(+), 60 deletions(-)

diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLevelLoader.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLevelLoader.java
index e4ffe2a5e8..88920c05fc 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLevelLoader.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLevelLoader.java
@@ -20,7 +20,8 @@ package org.apache.paimon.flink.lookup;
 
 import org.apache.paimon.data.BinaryRow;
 import org.apache.paimon.data.InternalRow;
-import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.table.Table;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.Preconditions;
 
@@ -34,6 +35,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
+import static org.apache.paimon.CoreOptions.PARTITION_DEFAULT_NAME;
+
 /** Dynamic partition loader which can specify the partition level to load for 
lookup. */
 public class DynamicPartitionLevelLoader extends DynamicPartitionLoader {
 
@@ -47,14 +50,12 @@ public class DynamicPartitionLevelLoader extends 
DynamicPartitionLoader {
     private final String defaultPartitionName;
 
     DynamicPartitionLevelLoader(
-            FileStoreTable table,
-            Duration refreshInterval,
-            Map<String, String> partitionLoadConfig) {
+            Table table, Duration refreshInterval, Map<String, String> 
partitionLoadConfig) {
         super(table, refreshInterval);
         maxPartitionLoadLevel =
                 getMaxPartitionLoadLevel(partitionLoadConfig, 
table.partitionKeys());
         fieldGetters = createPartitionFieldGetters();
-        defaultPartitionName = table.coreOptions().partitionDefaultName();
+        defaultPartitionName = 
Options.fromMap(table.options()).get(PARTITION_DEFAULT_NAME);
 
         LOG.info(
                 "Init 
DynamicPartitionLevelLoader(table={}),maxPartitionLoadLevel is {}",
@@ -63,7 +64,7 @@ public class DynamicPartitionLevelLoader extends 
DynamicPartitionLoader {
     }
 
     @Override
-    protected List<BinaryRow> getMaxPartitions() {
+    public List<BinaryRow> getMaxPartitions() {
         List<BinaryRow> newPartitions =
                 table.newReadBuilder().newScan().listPartitions().stream()
                         .sorted(comparator.reversed())
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java
index 201997da55..37671dee0f 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java
@@ -21,7 +21,7 @@ package org.apache.paimon.flink.lookup;
 import org.apache.paimon.codegen.CodeGenUtils;
 import org.apache.paimon.data.BinaryRow;
 import org.apache.paimon.data.InternalRow;
-import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.Table;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.InternalRowPartitionComputer;
 
@@ -46,7 +46,7 @@ public abstract class DynamicPartitionLoader extends 
PartitionLoader {
     protected transient Comparator<InternalRow> comparator;
     protected transient LocalDateTime lastRefresh;
 
-    DynamicPartitionLoader(FileStoreTable table, Duration refreshInterval) {
+    DynamicPartitionLoader(Table table, Duration refreshInterval) {
         super(table);
         this.refreshInterval = refreshInterval;
     }
@@ -92,7 +92,7 @@ public abstract class DynamicPartitionLoader extends 
PartitionLoader {
         }
     }
 
-    protected abstract List<BinaryRow> getMaxPartitions();
+    public abstract List<BinaryRow> getMaxPartitions();
 
     private void logNewPartitions() {
         String partitionsStr = partitionsToString(partitions);
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionNumberLoader.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionNumberLoader.java
index 52c1d4c76c..7e210725de 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionNumberLoader.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionNumberLoader.java
@@ -19,7 +19,7 @@
 package org.apache.paimon.flink.lookup;
 
 import org.apache.paimon.data.BinaryRow;
-import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.Table;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -37,8 +37,7 @@ public class DynamicPartitionNumberLoader extends 
DynamicPartitionLoader {
 
     private final int maxPartitionNum;
 
-    DynamicPartitionNumberLoader(
-            FileStoreTable table, Duration refreshInterval, int 
maxPartitionNum) {
+    DynamicPartitionNumberLoader(Table table, Duration refreshInterval, int 
maxPartitionNum) {
         super(table, refreshInterval);
         this.maxPartitionNum = maxPartitionNum;
         LOG.info(
@@ -47,7 +46,8 @@ public class DynamicPartitionNumberLoader extends 
DynamicPartitionLoader {
                 maxPartitionNum);
     }
 
-    protected List<BinaryRow> getMaxPartitions() {
+    @Override
+    public List<BinaryRow> getMaxPartitions() {
         List<BinaryRow> newPartitions =
                 table.newReadBuilder().newScan().listPartitions().stream()
                         .sorted(comparator.reversed())
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
index 7df4742101..121a86d782 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java
@@ -227,7 +227,7 @@ public class FileStoreLookupFunction implements 
Serializable, Closeable {
             List<BinaryRow> partitions = partitionLoader.partitions();
             if (!partitions.isEmpty()) {
                 lookupTable.specifyPartitions(
-                        partitions, 
partitionLoader.createSpecificPartFilter());
+                        partitions, 
partitionLoader.createSpecificPartFilter(partitions));
             }
         }
 
@@ -329,7 +329,7 @@ public class FileStoreLookupFunction implements 
Serializable, Closeable {
             if (partitionChanged) {
                 // reopen with latest partition
                 lookupTable.specifyPartitions(
-                        partitionLoader.partitions(), 
partitionLoader.createSpecificPartFilter());
+                        partitions, 
partitionLoader.createSpecificPartFilter(partitions));
                 lookupTable.close();
                 lookupTable.open();
                 // no need to refresh the lookup table because it is reopened
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionLoader.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionLoader.java
index 701c40b67c..72f7777f01 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionLoader.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PartitionLoader.java
@@ -24,7 +24,7 @@ import org.apache.paimon.options.Options;
 import org.apache.paimon.partition.PartitionPredicate;
 import org.apache.paimon.predicate.Predicate;
 import org.apache.paimon.predicate.PredicateBuilder;
-import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.Table;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.ParameterUtils;
 import org.apache.paimon.utils.Preconditions;
@@ -44,15 +44,15 @@ public abstract class PartitionLoader implements 
Serializable {
 
     private static final long serialVersionUID = 1L;
 
-    protected static final String MAX_PT = "max_pt()";
-    protected static final String MAX_TWO_PT = "max_two_pt()";
+    private static final String MAX_PT = "max_pt()";
+    private static final String MAX_TWO_PT = "max_two_pt()";
 
-    protected final FileStoreTable table;
+    protected final Table table;
     private final RowDataToObjectArrayConverter partitionConverter;
 
     protected transient List<BinaryRow> partitions;
 
-    protected PartitionLoader(FileStoreTable table) {
+    protected PartitionLoader(Table table) {
         this.table = table;
         this.partitionConverter =
                 new 
RowDataToObjectArrayConverter(table.rowType().project(table.partitionKeys()));
@@ -76,7 +76,7 @@ public abstract class PartitionLoader implements Serializable 
{
         partitionKeys.stream().filter(k -> 
!projectFields.contains(k)).forEach(projectFields::add);
     }
 
-    public Predicate createSpecificPartFilter() {
+    public Predicate createSpecificPartFilter(List<BinaryRow> partitions) {
         Predicate partFilter = null;
         for (BinaryRow partition : partitions) {
             if (partFilter == null) {
@@ -105,7 +105,7 @@ public abstract class PartitionLoader implements 
Serializable {
     public abstract boolean checkRefresh();
 
     @Nullable
-    public static PartitionLoader of(FileStoreTable table) {
+    public static PartitionLoader of(Table table) {
         Options options = Options.fromMap(table.options());
         String scanPartitions = 
options.get(FlinkConnectorOptions.SCAN_PARTITIONS);
         if (scanPartitions == null) {
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/StaticPartitionLoader.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/StaticPartitionLoader.java
index b6387a20a5..bdfecc39f9 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/StaticPartitionLoader.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/StaticPartitionLoader.java
@@ -20,7 +20,8 @@ package org.apache.paimon.flink.lookup;
 
 import org.apache.paimon.data.GenericRow;
 import org.apache.paimon.data.serializer.InternalRowSerializer;
-import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.table.Table;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.InternalRowPartitionComputer;
 
@@ -28,13 +29,14 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.paimon.CoreOptions.PARTITION_DEFAULT_NAME;
+
 /** {@link PartitionLoader} for specified static partitions. */
 public class StaticPartitionLoader extends PartitionLoader {
 
     private final List<Map<String, String>> scanPartitions;
 
-    protected StaticPartitionLoader(
-            FileStoreTable table, List<Map<String, String>> scanPartitions) {
+    protected StaticPartitionLoader(Table table, List<Map<String, String>> 
scanPartitions) {
         super(table);
         this.scanPartitions = scanPartitions;
     }
@@ -42,12 +44,13 @@ public class StaticPartitionLoader extends PartitionLoader {
     @Override
     public void open() {
         partitions = new ArrayList<>();
-        RowType partitionType = table.schema().logicalPartitionType();
+        RowType partitionType = table.rowType().project(table.partitionKeys());
+        String defaultPartitionName = 
Options.fromMap(table.options()).get(PARTITION_DEFAULT_NAME);
         InternalRowSerializer serializer = new 
InternalRowSerializer(partitionType);
         for (Map<String, String> spec : scanPartitions) {
             GenericRow row =
                     InternalRowPartitionComputer.convertSpecToInternalRow(
-                            spec, partitionType, 
table.coreOptions().partitionDefaultName());
+                            spec, partitionType, defaultPartitionName);
             partitions.add(serializer.toBinaryRow(row).copy());
         }
     }
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java
index d0e79e2a05..2ee8f235d5 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java
@@ -19,9 +19,13 @@
 package org.apache.paimon.flink.source;
 
 import org.apache.paimon.CoreOptions;
+import org.apache.paimon.data.BinaryRow;
 import org.apache.paimon.flink.FlinkConnectorOptions;
 import org.apache.paimon.flink.LogicalTypeConversion;
 import org.apache.paimon.flink.PredicateConverter;
+import org.apache.paimon.flink.lookup.DynamicPartitionLoader;
+import org.apache.paimon.flink.lookup.PartitionLoader;
+import org.apache.paimon.flink.lookup.StaticPartitionLoader;
 import org.apache.paimon.manifest.PartitionEntry;
 import org.apache.paimon.options.Options;
 import org.apache.paimon.partition.PartitionPredicate;
@@ -30,9 +34,9 @@ import org.apache.paimon.predicate.Predicate;
 import org.apache.paimon.predicate.PredicateBuilder;
 import org.apache.paimon.predicate.PredicateVisitor;
 import org.apache.paimon.table.DataTable;
+import org.apache.paimon.table.FileStoreTable;
 import org.apache.paimon.table.Table;
 import org.apache.paimon.table.source.Split;
-import org.apache.paimon.utils.ParameterUtils;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -52,6 +56,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Optional;
 
+import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_PARTITIONS;
 import static org.apache.paimon.options.OptionsUtils.PAIMON_PREFIX;
 import static 
org.apache.paimon.predicate.PredicateBuilder.transformFieldMapping;
 
@@ -137,43 +142,42 @@ public abstract class FlinkTableSource
 
     /**
      * This method is only used for normal source (not lookup source). 
Specified partitions in
-     * lookup sources are handled in {@link 
org.apache.paimon.flink.lookup.PartitionLoader}.
+     * lookup sources are handled in {@link 
org.apache.paimon.flink.lookup.PartitionLoader}. But we
+     * use PartitionLoader to create partition predicate. It's possible that 
user use max_pt() or
+     * max_two_pt() in batch join, so we should also handle it.
      */
     private PartitionPredicate getPartitionPredicateWithOptions() {
-        if (options.contains(FlinkConnectorOptions.SCAN_PARTITIONS)) {
-            try {
-                Predicate predicate =
-                        PartitionPredicate.createPartitionPredicate(
-                                ParameterUtils.getPartitions(
-                                        
options.get(FlinkConnectorOptions.SCAN_PARTITIONS)
-                                                .split(";")),
-                                table.rowType(),
-                                
options.get(CoreOptions.PARTITION_DEFAULT_NAME));
-                // Partition filter will be used to filter Manifest stats, the 
stats schema is
-                // partition type. See SnapshotReaderImpl#withFilter
-                Predicate transformed =
-                        transformFieldMapping(
-                                        predicate,
-                                        
PredicateBuilder.fieldIdxToPartitionIdx(
-                                                table.rowType(), 
table.partitionKeys()))
-                                .orElseThrow(
-                                        () ->
-                                                new RuntimeException(
-                                                        "Failed to transform 
the partition predicate "
-                                                                + predicate));
-                return PartitionPredicate.fromPredicate(
-                        table.rowType().project(table.partitionKeys()), 
transformed);
-            } catch (IllegalArgumentException e) {
-                // In older versions of Flink, however, lookup sources will 
first be treated as
-                // normal sources. So this method will also be visited by 
lookup tables, whose
-                // option value might be max_pt() or max_two_pt(). In this 
case we ignore the
-                // filters.
-                return null;
-            }
+        PartitionLoader partitionLoader = PartitionLoader.of((FileStoreTable) 
table);
+        if (partitionLoader == null) {
+            return null;
+        }
 
+        partitionLoader.open();
+        List<BinaryRow> partitions;
+        if (partitionLoader instanceof StaticPartitionLoader) {
+            partitions = partitionLoader.partitions();
+        } else if (partitionLoader instanceof DynamicPartitionLoader) {
+            partitions = ((DynamicPartitionLoader) 
partitionLoader).getMaxPartitions();
         } else {
-            return null;
+            throw new RuntimeException(
+                    "Failed to handle scan.partitions = " + 
options.get(SCAN_PARTITIONS));
         }
+
+        // Partition filter will be used to filter Manifest stats, the stats 
schema is
+        // partition type. See SnapshotReaderImpl#withFilter
+        Predicate predicate = 
partitionLoader.createSpecificPartFilter(partitions);
+        Predicate transformed =
+                transformFieldMapping(
+                                predicate,
+                                PredicateBuilder.fieldIdxToPartitionIdx(
+                                        table.rowType(), 
table.partitionKeys()))
+                        .orElseThrow(
+                                () ->
+                                        new RuntimeException(
+                                                "Failed to transform the 
partition predicate "
+                                                        + predicate));
+        return PartitionPredicate.fromPredicate(
+                table.rowType().project(table.partitionKeys()), transformed);
     }
 
     @Override
diff --git 
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java
 
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java
index 589eeb14af..a81cf7a506 100644
--- 
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java
+++ 
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java
@@ -902,6 +902,46 @@ public class BatchFileStoreITCase extends 
CatalogITCaseBase {
         assertThat(sql(query)).containsExactly(Row.of(1, 11), Row.of(1, 12), 
Row.of(2, 22));
     }
 
+    @Test
+    public void testScanWithSpecifiedPartitionsWithMaxPt() {
+        sql("CREATE TABLE P (id INT, v INT, pt STRING) PARTITIONED BY (pt)");
+        sql("CREATE TABLE Q (id INT)");
+        sql(
+                "INSERT INTO P VALUES (1, 10, 'a'), (2, 20, 'a'), (1, 11, 
'b'), (3, 31, 'b'), (1, 12, 'c'), (2, 22, 'c'), (3, 32, 'c')");
+        sql("INSERT INTO Q VALUES (1), (2), (3)");
+        String query =
+                "SELECT Q.id, P.v FROM Q INNER JOIN P /*+ 
OPTIONS('scan.partitions' = 'max_pt()') */ ON Q.id = P.id ORDER BY Q.id, P.v";
+        assertThat(sql(query)).containsExactly(Row.of(1, 12), Row.of(2, 22), 
Row.of(3, 32));
+    }
+
+    @Test
+    public void testScanWithSpecifiedPartitionsWithMaxTwoPt() {
+        sql("CREATE TABLE P (id INT, v INT, pt STRING) PARTITIONED BY (pt)");
+        sql("CREATE TABLE Q (id INT)");
+        sql(
+                "INSERT INTO P VALUES (1, 10, 'a'), (2, 20, 'a'), (1, 11, 
'b'), (3, 31, 'b'), (1, 12, 'c'), (2, 22, 'c'), (3, 32, 'c')");
+        sql("INSERT INTO Q VALUES (1), (2), (3)");
+        String query =
+                "SELECT Q.id, P.v FROM Q INNER JOIN P /*+ 
OPTIONS('scan.partitions' = 'max_two_pt()') */ ON Q.id = P.id ORDER BY Q.id, 
P.v";
+        assertThat(sql(query))
+                .containsExactly(
+                        Row.of(1, 11), Row.of(1, 12), Row.of(2, 22), Row.of(3, 
31), Row.of(3, 32));
+    }
+
+    @Test
+    public void testScanWithSpecifiedPartitionsWithLevelMaxPt() throws 
Exception {
+        sql(
+                "CREATE TABLE P (id INT, v INT, pt1 STRING, pt2 STRING, pt3 
STRING) PARTITIONED BY (pt1, pt2, pt3)");
+        sql("CREATE TABLE Q (id INT)");
+        sql(
+                "INSERT INTO P VALUES (1, 10, 'a', '2025-10-01', '1'), (2, 20, 
'a', '2025-10-01', '2'), (3, 30, 'a', '2025-10-02', '1'), (4, 40, 'a', 
'2025-10-02', '2'), "
+                        + "(1, 11, 'b', '2025-10-01', '1'), (2, 21, 'b', 
'2025-10-01', '2'), (3, 31, 'b', '2025-10-02', '1'), (4, 41, 'b', '2025-10-02', 
'2')");
+        sql("INSERT INTO Q VALUES (1), (2), (3), (4)");
+        String query =
+                "SELECT Q.id, P.v FROM Q INNER JOIN P /*+ 
OPTIONS('scan.partitions' = 'pt1=max_pt(),pt2=max_pt()') */ ON Q.id = P.id 
ORDER BY Q.id, P.v";
+        assertThat(sql(query)).containsExactly(Row.of(3, 31), Row.of(4, 41));
+    }
+
     @Test
     public void testEmptyTableIncrementalBetweenTimestamp() {
         assertThat(sql("SELECT * FROM T /*+ 
OPTIONS('incremental-between-timestamp'='0,1') */"))

Reply via email to