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

lzljs3620320 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 cf6fb60c29 [core] Introduce global index enabled scan (#6794)
cf6fb60c29 is described below

commit cf6fb60c29d9e6242a8e252580ed5d48de5e21bb
Author: YeJunHao <[email protected]>
AuthorDate: Thu Dec 11 21:10:44 2025 +0800

    [core] Introduce global index enabled scan (#6794)
---
 .../shortcodes/generated/core_configuration.html   |  12 +
 .../main/java/org/apache/paimon/CoreOptions.java   |  22 ++
 .../paimon/globalindex/GlobalIndexResult.java      |  12 +
 .../apache/paimon/globalindex/GlobalIndexer.java   |   6 +-
 .../paimon/globalindex/GlobalIndexerFactory.java   |   4 +-
 ...{GlobalIndexerFactory.java => ScoreGetter.java} |  12 +-
 ...exerFactory.java => TopkGlobalIndexResult.java} |  12 +-
 .../bitmap/BitmapGlobalIndexerFactory.java         |   6 +-
 .../java/org/apache/paimon/utils/FloatUtils.java   |  45 ++--
 .../apache/paimon/utils/RoaringNavigableMap64.java |   6 +-
 .../java/org/apache/paimon/AbstractFileStore.java  |  13 ++
 .../src/main/java/org/apache/paimon/FileStore.java |   3 +
 .../paimon/globalindex/GlobalIndexBatchScan.java   | 244 +++++++++++++++++++++
 .../paimon/globalindex/GlobalIndexScanBuilder.java |  65 +++++-
 .../globalindex/GlobalIndexScanBuilderImpl.java    | 148 +++++++++----
 .../apache/paimon/globalindex/IndexedSplit.java    |   3 +-
 .../globalindex/RowRangeGlobalIndexScanner.java    |  35 ++-
 .../paimon/operation/DataEvolutionSplitRead.java   |   9 +-
 .../apache/paimon/operation/RawFileSplitRead.java  |  16 --
 .../org/apache/paimon/operation/SplitRead.java     |  12 -
 .../paimon/privilege/PrivilegedFileStore.java      |   6 +
 .../org/apache/paimon/table/FileStoreTable.java    |   3 +-
 .../paimon/table/source/AbstractDataTableRead.java |  10 -
 .../paimon/table/source/AbstractDataTableScan.java |   6 +
 .../paimon/table/source/AppendTableRead.java       |  14 +-
 .../apache/paimon/table/source/InnerTableRead.java |   5 -
 .../apache/paimon/table/source/InnerTableScan.java |   5 +
 .../paimon/table/source/KeyValueTableRead.java     |   6 -
 .../paimon/table/source/ReadBuilderImpl.java       |  20 +-
 .../paimon/table/DataEvolutionTableTest.java       | 156 +++++++------
 .../paimon/flink/lookup/LookupCompactDiffRead.java |   7 -
 .../spark/globalindex/GlobalIndexBuilder.java      |  15 +-
 .../globalindex/GlobalIndexBuilderContext.java     |  11 +-
 .../procedure/CreateGlobalIndexProcedure.java      |  36 +--
 .../procedure/CreateGlobalIndexProcedureTest.java  |  85 ++++---
 35 files changed, 746 insertions(+), 324 deletions(-)

diff --git a/docs/layouts/shortcodes/generated/core_configuration.html 
b/docs/layouts/shortcodes/generated/core_configuration.html
index 9b2bf644fc..3abaf417b1 100644
--- a/docs/layouts/shortcodes/generated/core_configuration.html
+++ b/docs/layouts/shortcodes/generated/core_configuration.html
@@ -590,12 +590,24 @@ under the License.
             <td>Integer</td>
             <td>For streaming write, full compaction will be constantly 
triggered after delta commits. For batch write, full compaction will be 
triggered with each commit as long as this value is greater than 0.</td>
         </tr>
+        <tr>
+            <td><h5>global-index.enabled</h5></td>
+            <td style="word-wrap: break-word;">true</td>
+            <td>Boolean</td>
+            <td>Whether to enable global index for scan.</td>
+        </tr>
         <tr>
             <td><h5>global-index.row-count-per-shard</h5></td>
             <td style="word-wrap: break-word;">100000</td>
             <td>Long</td>
             <td>Row count per shard for global index.</td>
         </tr>
+        <tr>
+            <td><h5>global-index.thread-num</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>Integer</td>
+            <td>The maximum number of concurrent scanner for global index.By 
default is the number of processors available to the Java virtual machine.</td>
+        </tr>
         <tr>
             <td><h5>ignore-delete</h5></td>
             <td style="word-wrap: break-word;">false</td>
diff --git a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java 
b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java
index 19b44d2a2a..0151ad96e9 100644
--- a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java
+++ b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java
@@ -2098,6 +2098,20 @@ public class CoreOptions implements Serializable {
                     .defaultValue(100000L)
                     .withDescription("Row count per shard for global index.");
 
+    public static final ConfigOption<Boolean> GLOBAL_INDEX_ENABLED =
+            key("global-index.enabled")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription("Whether to enable global index for 
scan.");
+
+    public static final ConfigOption<Integer> GLOBAL_INDEX_THREAD_NUM =
+            key("global-index.thread-num")
+                    .intType()
+                    .noDefaultValue()
+                    .withDescription(
+                            "The maximum number of concurrent scanner for 
global index."
+                                    + "By default is the number of processors 
available to the Java virtual machine.");
+
     private final Options options;
 
     public CoreOptions(Map<String, String> options) {
@@ -3251,6 +3265,14 @@ public class CoreOptions implements Serializable {
         return options.get(GLOBAL_INDEX_ROW_COUNT_PER_SHARD);
     }
 
+    public boolean globalIndexEnabled() {
+        return options.get(GLOBAL_INDEX_ENABLED);
+    }
+
+    public Integer globalIndexThreadNum() {
+        return options.get(GLOBAL_INDEX_THREAD_NUM);
+    }
+
     /** Specifies the merge engine for table with primary key. */
     public enum MergeEngine implements DescribedEnum {
         DEDUPLICATE("deduplicate", "De-duplicate and keep the last row."),
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexResult.java
 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexResult.java
index e5ad57f336..6b708fb7a4 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexResult.java
+++ 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexResult.java
@@ -22,6 +22,7 @@ import org.apache.paimon.utils.LazyField;
 import org.apache.paimon.utils.Range;
 import org.apache.paimon.utils.RoaringNavigableMap64;
 
+import java.util.List;
 import java.util.function.Supplier;
 
 /** Global index result represents row ids as a compressed bitmap. */
@@ -68,4 +69,15 @@ public interface GlobalIndexResult {
                     return result64;
                 });
     }
+
+    static GlobalIndexResult fromRanges(List<Range> ranges) {
+        return create(
+                () -> {
+                    RoaringNavigableMap64 result64 = new 
RoaringNavigableMap64();
+                    for (Range range : ranges) {
+                        result64.addRange(range);
+                    }
+                    return result64;
+                });
+    }
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexer.java 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexer.java
index 2ca5fb31af..33ea405a8f 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexer.java
+++ 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexer.java
@@ -21,7 +21,7 @@ package org.apache.paimon.globalindex;
 import org.apache.paimon.globalindex.io.GlobalIndexFileReader;
 import org.apache.paimon.globalindex.io.GlobalIndexFileWriter;
 import org.apache.paimon.options.Options;
-import org.apache.paimon.types.DataType;
+import org.apache.paimon.types.DataField;
 
 import java.io.IOException;
 import java.util.List;
@@ -34,8 +34,8 @@ public interface GlobalIndexer {
     GlobalIndexReader createReader(GlobalIndexFileReader fileReader, 
List<GlobalIndexIOMeta> files)
             throws IOException;
 
-    static GlobalIndexer create(String type, DataType dataType, Options 
options) {
+    static GlobalIndexer create(String type, DataField dataField, Options 
options) {
         GlobalIndexerFactory globalIndexerFactory = 
GlobalIndexerFactoryUtils.load(type);
-        return globalIndexerFactory.create(dataType, options);
+        return globalIndexerFactory.create(dataField, options);
     }
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
index 53154ef50b..6eabb6d253 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
+++ 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
@@ -20,12 +20,12 @@ package org.apache.paimon.globalindex;
 
 import org.apache.paimon.fileindex.FileIndexer;
 import org.apache.paimon.options.Options;
-import org.apache.paimon.types.DataType;
+import org.apache.paimon.types.DataField;
 
 /** File index factory to construct {@link FileIndexer}. */
 public interface GlobalIndexerFactory {
 
     String identifier();
 
-    GlobalIndexer create(DataType type, Options options);
+    GlobalIndexer create(DataField dataField, Options options);
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
 b/paimon-common/src/main/java/org/apache/paimon/globalindex/ScoreGetter.java
similarity index 72%
copy from 
paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
copy to 
paimon-common/src/main/java/org/apache/paimon/globalindex/ScoreGetter.java
index 53154ef50b..c43b432301 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
+++ b/paimon-common/src/main/java/org/apache/paimon/globalindex/ScoreGetter.java
@@ -18,14 +18,8 @@
 
 package org.apache.paimon.globalindex;
 
-import org.apache.paimon.fileindex.FileIndexer;
-import org.apache.paimon.options.Options;
-import org.apache.paimon.types.DataType;
+/** Generate score (may null) for row id in global index top-k situation. */
+public interface ScoreGetter {
 
-/** File index factory to construct {@link FileIndexer}. */
-public interface GlobalIndexerFactory {
-
-    String identifier();
-
-    GlobalIndexer create(DataType type, Options options);
+    float score(long rowId);
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/TopkGlobalIndexResult.java
similarity index 72%
copy from 
paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
copy to 
paimon-common/src/main/java/org/apache/paimon/globalindex/TopkGlobalIndexResult.java
index 53154ef50b..12e15260f4 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/GlobalIndexerFactory.java
+++ 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/TopkGlobalIndexResult.java
@@ -18,14 +18,8 @@
 
 package org.apache.paimon.globalindex;
 
-import org.apache.paimon.fileindex.FileIndexer;
-import org.apache.paimon.options.Options;
-import org.apache.paimon.types.DataType;
+/** Top-k global index result for vector index. */
+public interface TopkGlobalIndexResult extends GlobalIndexResult {
 
-/** File index factory to construct {@link FileIndexer}. */
-public interface GlobalIndexerFactory {
-
-    String identifier();
-
-    GlobalIndexer create(DataType type, Options options);
+    ScoreGetter scoreGetter();
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/bitmap/BitmapGlobalIndexerFactory.java
 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/bitmap/BitmapGlobalIndexerFactory.java
index efbe0bbd2f..2150b3072a 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/globalindex/bitmap/BitmapGlobalIndexerFactory.java
+++ 
b/paimon-common/src/main/java/org/apache/paimon/globalindex/bitmap/BitmapGlobalIndexerFactory.java
@@ -22,7 +22,7 @@ import org.apache.paimon.fileindex.bitmap.BitmapFileIndex;
 import org.apache.paimon.globalindex.GlobalIndexer;
 import org.apache.paimon.globalindex.GlobalIndexerFactory;
 import org.apache.paimon.options.Options;
-import org.apache.paimon.types.DataType;
+import org.apache.paimon.types.DataField;
 
 /** Factory for creating bitmap global indexers. */
 public class BitmapGlobalIndexerFactory implements GlobalIndexerFactory {
@@ -35,8 +35,8 @@ public class BitmapGlobalIndexerFactory implements 
GlobalIndexerFactory {
     }
 
     @Override
-    public GlobalIndexer create(DataType type, Options options) {
-        BitmapFileIndex bitmapFileIndex = new BitmapFileIndex(type, options);
+    public GlobalIndexer create(DataField dataField, Options options) {
+        BitmapFileIndex bitmapFileIndex = new 
BitmapFileIndex(dataField.type(), options);
         return new BitmapGlobalIndex(bitmapFileIndex);
     }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilder.java
 b/paimon-common/src/main/java/org/apache/paimon/utils/FloatUtils.java
similarity index 57%
copy from 
paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilder.java
copy to paimon-common/src/main/java/org/apache/paimon/utils/FloatUtils.java
index 22f55655ca..1c420c96d9 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilder.java
+++ b/paimon-common/src/main/java/org/apache/paimon/utils/FloatUtils.java
@@ -16,23 +16,30 @@
  * limitations under the License.
  */
 
-package org.apache.paimon.globalindex;
-
-import org.apache.paimon.data.BinaryRow;
-import org.apache.paimon.utils.Range;
-
-import java.util.Set;
-
-/** Builder for scanning global indexes. */
-public interface GlobalIndexScanBuilder {
-
-    GlobalIndexScanBuilder withSnapshot(long snapshotId);
-
-    GlobalIndexScanBuilder withPartition(BinaryRow binaryRow);
-
-    GlobalIndexScanBuilder withRowRange(Range rowRange);
-
-    RowRangeGlobalIndexScanner build();
-
-    Set<Range> shardList();
+package org.apache.paimon.utils;
+
+/** Utils for float. */
+public class FloatUtils {
+
+    public static boolean equals(float[] arr1, float[] arr2, float epsilon) {
+        if (arr1 == arr2) {
+            return true;
+        }
+
+        if (arr1 == null || arr2 == null) {
+            return false;
+        }
+
+        if (arr1.length != arr2.length) {
+            return false;
+        }
+
+        for (int i = 0; i < arr1.length; i++) {
+            float diff = Math.abs(arr1[i] - arr2[i]);
+            if (diff > epsilon) {
+                return false;
+            }
+        }
+        return true;
+    }
 }
diff --git 
a/paimon-common/src/main/java/org/apache/paimon/utils/RoaringNavigableMap64.java
 
b/paimon-common/src/main/java/org/apache/paimon/utils/RoaringNavigableMap64.java
index 0765ab1e78..6bb52af3d3 100644
--- 
a/paimon-common/src/main/java/org/apache/paimon/utils/RoaringNavigableMap64.java
+++ 
b/paimon-common/src/main/java/org/apache/paimon/utils/RoaringNavigableMap64.java
@@ -46,7 +46,11 @@ public class RoaringNavigableMap64 {
     }
 
     public void addRange(Range range) {
-        roaring64NavigableMap.addRange(range.from, range.to);
+        if (range.from == range.to) {
+            roaring64NavigableMap.add(range.from);
+        } else {
+            roaring64NavigableMap.addRange(range.from, range.to);
+        }
     }
 
     public void add(long x) {
diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java 
b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
index a7e15b31d6..baf03f514f 100644
--- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
@@ -25,6 +25,8 @@ import org.apache.paimon.data.InternalRow;
 import org.apache.paimon.format.FileFormat;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.fs.Path;
+import org.apache.paimon.globalindex.GlobalIndexScanBuilder;
+import org.apache.paimon.globalindex.GlobalIndexScanBuilderImpl;
 import org.apache.paimon.iceberg.IcebergCommitCallback;
 import org.apache.paimon.iceberg.IcebergOptions;
 import org.apache.paimon.index.IndexFileHandler;
@@ -494,4 +496,15 @@ abstract class AbstractFileStore<T> implements 
FileStore<T> {
     public void setSnapshotCache(Cache<Path, Snapshot> cache) {
         this.snapshotCache = cache;
     }
+
+    @Override
+    public GlobalIndexScanBuilder newIndexScanBuilder() {
+        return new GlobalIndexScanBuilderImpl(
+                options.toConfiguration(),
+                schema.logicalRowType(),
+                fileIO,
+                pathFactory().globalIndexFileFactory(),
+                snapshotManager(),
+                newIndexFileHandler());
+    }
 }
diff --git a/paimon-core/src/main/java/org/apache/paimon/FileStore.java 
b/paimon-core/src/main/java/org/apache/paimon/FileStore.java
index 98905b47e5..b73b9276a3 100644
--- a/paimon-core/src/main/java/org/apache/paimon/FileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/FileStore.java
@@ -19,6 +19,7 @@
 package org.apache.paimon;
 
 import org.apache.paimon.fs.Path;
+import org.apache.paimon.globalindex.GlobalIndexScanBuilder;
 import org.apache.paimon.index.IndexFileHandler;
 import org.apache.paimon.manifest.IndexManifestFile;
 import org.apache.paimon.manifest.ManifestFile;
@@ -124,4 +125,6 @@ public interface FileStore<T> {
     void setManifestCache(SegmentsCache<Path> manifestCache);
 
     void setSnapshotCache(Cache<Path, Snapshot> cache);
+
+    GlobalIndexScanBuilder newIndexScanBuilder();
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexBatchScan.java
 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexBatchScan.java
new file mode 100644
index 0000000000..264edea6f0
--- /dev/null
+++ 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexBatchScan.java
@@ -0,0 +1,244 @@
+/*
+ * 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.globalindex;
+
+import org.apache.paimon.Snapshot;
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.io.DataFileMeta;
+import org.apache.paimon.manifest.PartitionEntry;
+import org.apache.paimon.metrics.MetricRegistry;
+import org.apache.paimon.partition.PartitionPredicate;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.predicate.TopN;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.source.DataSplit;
+import org.apache.paimon.table.source.InnerTableScan;
+import org.apache.paimon.table.source.Split;
+import org.apache.paimon.table.source.snapshot.TimeTravelUtil;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.Filter;
+import org.apache.paimon.utils.Range;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static 
org.apache.paimon.globalindex.GlobalIndexScanBuilder.parallelScan;
+
+/** Scan with global index. */
+public class GlobalIndexBatchScan implements InnerTableScan {
+    private final FileStoreTable wrapped;
+    private final InnerTableScan batchScan;
+    private GlobalIndexResult globalIndexResult;
+    private Predicate filter;
+
+    public GlobalIndexBatchScan(FileStoreTable wrapped, InnerTableScan 
batchScan) {
+        this.wrapped = wrapped;
+        this.batchScan = batchScan;
+    }
+
+    @Override
+    public InnerTableScan withFilter(Predicate predicate) {
+        this.filter = predicate;
+        batchScan.withFilter(predicate);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withReadType(@Nullable RowType readType) {
+        batchScan.withReadType(readType);
+        return this;
+    }
+
+    @Nullable
+    @Override
+    public PartitionPredicate partitionFilter() {
+        return batchScan.partitionFilter();
+    }
+
+    @Override
+    public InnerTableScan withBucket(int bucket) {
+        batchScan.withBucket(bucket);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withTopN(TopN topN) {
+        batchScan.withTopN(topN);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan dropStats() {
+        batchScan.dropStats();
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withMetricRegistry(MetricRegistry metricsRegistry) {
+        batchScan.withMetricRegistry(metricsRegistry);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withLimit(int limit) {
+        batchScan.withLimit(limit);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withPartitionFilter(Map<String, String> 
partitionSpec) {
+        batchScan.withPartitionFilter(partitionSpec);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withPartitionFilter(List<BinaryRow> partitions) {
+        batchScan.withPartitionFilter(partitions);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withPartitionsFilter(List<Map<String, String>> 
partitions) {
+        batchScan.withPartitionsFilter(partitions);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withPartitionFilter(PartitionPredicate 
partitionPredicate) {
+        batchScan.withPartitionFilter(partitionPredicate);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withBucketFilter(Filter<Integer> bucketFilter) {
+        batchScan.withBucketFilter(bucketFilter);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withLevelFilter(Filter<Integer> levelFilter) {
+        batchScan.withLevelFilter(levelFilter);
+        return this;
+    }
+
+    @Override
+    public InnerTableScan withRowRanges(List<Range> rowRanges) {
+        if (rowRanges != null) {
+            this.globalIndexResult = GlobalIndexResult.fromRanges(rowRanges);
+        }
+        return this;
+    }
+
+    public InnerTableScan withGlobalIndexResult(GlobalIndexResult 
globalIndexResult) {
+        this.globalIndexResult = globalIndexResult;
+        return this;
+    }
+
+    private void configureGlobalIndex(InnerTableScan scan) {
+        if (globalIndexResult == null && filter != null) {
+            PartitionPredicate partitionPredicate = scan.partitionFilter();
+            GlobalIndexScanBuilder globalIndexScanBuilder = 
wrapped.newIndexScanBuilder();
+            Snapshot snapshot = TimeTravelUtil.tryTravelOrLatest(wrapped);
+            globalIndexScanBuilder
+                    .withPartitionPredicate(partitionPredicate)
+                    .withSnapshot(snapshot);
+            List<Range> indexedRowRanges = globalIndexScanBuilder.shardList();
+            if (!indexedRowRanges.isEmpty()) {
+                List<Range> nonIndexedRowRanges =
+                        new Range(0, snapshot.nextRowId() - 
1).exclude(indexedRowRanges);
+                Optional<GlobalIndexResult> combined =
+                        parallelScan(
+                                indexedRowRanges,
+                                globalIndexScanBuilder,
+                                filter,
+                                wrapped.coreOptions().globalIndexThreadNum());
+                if (combined.isPresent()) {
+                    GlobalIndexResult globalIndexResultTemp = combined.get();
+                    if (!nonIndexedRowRanges.isEmpty()) {
+                        for (Range range : nonIndexedRowRanges) {
+                            
globalIndexResultTemp.or(GlobalIndexResult.fromRange(range));
+                        }
+                    }
+
+                    this.globalIndexResult = globalIndexResultTemp;
+                }
+            }
+        }
+
+        if (this.globalIndexResult != null) {
+            scan.withRowRanges(this.globalIndexResult.results().toRangeList());
+        }
+    }
+
+    @Override
+    public Plan plan() {
+        configureGlobalIndex(batchScan);
+        List<Split> splits = batchScan.plan().splits();
+        return wrap(splits);
+    }
+
+    private Plan wrap(List<Split> splits) {
+        if (globalIndexResult == null) {
+            return () -> splits;
+        }
+        List<Split> indexedSplits = new ArrayList<>();
+        for (Split split : splits) {
+            DataSplit dataSplit = (DataSplit) split;
+            List<Range> fromDataFile = new ArrayList<>();
+            for (DataFileMeta d : dataSplit.dataFiles()) {
+                fromDataFile.add(
+                        new Range(d.nonNullFirstRowId(), d.nonNullFirstRowId() 
+ d.rowCount() - 1));
+            }
+
+            fromDataFile = Range.mergeSortedAsPossible(fromDataFile);
+
+            List<Range> expected =
+                    Range.and(fromDataFile, 
globalIndexResult.results().toRangeList());
+
+            float[] scores = null;
+            if (globalIndexResult instanceof TopkGlobalIndexResult) {
+                ScoreGetter scoreFunction =
+                        ((TopkGlobalIndexResult) 
globalIndexResult).scoreGetter();
+                if (scoreFunction != null) {
+                    int size = expected.stream().mapToInt(r -> (int) (r.to - 
r.from + 1)).sum();
+                    scores = new float[size];
+
+                    int index = 0;
+                    for (Range range : expected) {
+                        for (long i = range.from; i <= range.to; i++) {
+                            scores[index++] = scoreFunction.score(i);
+                        }
+                    }
+                }
+            }
+
+            indexedSplits.add(new IndexedSplit(dataSplit, expected, scores));
+        }
+        return () -> indexedSplits;
+    }
+
+    @Override
+    public List<PartitionEntry> listPartitionEntries() {
+        return batchScan.listPartitionEntries();
+    }
+}
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilder.java
 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilder.java
index 22f55655ca..5c4ec92f81 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilder.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilder.java
@@ -18,21 +18,78 @@
 
 package org.apache.paimon.globalindex;
 
-import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.Snapshot;
+import org.apache.paimon.partition.PartitionPredicate;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.utils.IOUtils;
 import org.apache.paimon.utils.Range;
 
-import java.util.Set;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static 
org.apache.paimon.utils.ManifestReadThreadPool.randomlyExecuteSequentialReturn;
 
 /** Builder for scanning global indexes. */
 public interface GlobalIndexScanBuilder {
 
     GlobalIndexScanBuilder withSnapshot(long snapshotId);
 
-    GlobalIndexScanBuilder withPartition(BinaryRow binaryRow);
+    GlobalIndexScanBuilder withSnapshot(Snapshot snapshot);
+
+    GlobalIndexScanBuilder withPartitionPredicate(PartitionPredicate 
partitionPredicate);
 
     GlobalIndexScanBuilder withRowRange(Range rowRange);
 
     RowRangeGlobalIndexScanner build();
 
-    Set<Range> shardList();
+    // Return sorted and no overlap ranges
+    List<Range> shardList();
+
+    static Optional<GlobalIndexResult> parallelScan(
+            final List<Range> ranges,
+            final GlobalIndexScanBuilder globalIndexScanBuilder,
+            final Predicate filter,
+            final Integer threadNum) {
+        List<RowRangeGlobalIndexScanner> scanners =
+                ranges.stream()
+                        .map(globalIndexScanBuilder::withRowRange)
+                        .map(GlobalIndexScanBuilder::build)
+                        .collect(Collectors.toList());
+
+        try {
+            List<Optional<GlobalIndexResult>> rowsResults = new ArrayList<>();
+            Iterator<Optional<GlobalIndexResult>> resultIterators =
+                    randomlyExecuteSequentialReturn(
+                            scanner -> {
+                                Optional<GlobalIndexResult> result = 
scanner.scan(filter);
+                                return Collections.singletonList(result);
+                            },
+                            scanners,
+                            threadNum);
+            while (resultIterators.hasNext()) {
+                rowsResults.add(resultIterators.next());
+            }
+            if (rowsResults.stream().noneMatch(Optional::isPresent)) {
+                return Optional.empty();
+            }
+
+            GlobalIndexResult globalIndexResult = 
GlobalIndexResult.createEmpty();
+
+            for (int i = 0; i < ranges.size(); i++) {
+                if (rowsResults.get(i).isPresent()) {
+                    globalIndexResult = 
globalIndexResult.or(rowsResults.get(i).get());
+                } else {
+                    globalIndexResult =
+                            
globalIndexResult.or(GlobalIndexResult.fromRange(ranges.get(i)));
+                }
+            }
+            return Optional.of(globalIndexResult);
+        } finally {
+            IOUtils.closeAllQuietly(scanners);
+        }
+    }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilderImpl.java
 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilderImpl.java
index 8791c3e8f1..3bf35acc47 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilderImpl.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/GlobalIndexScanBuilderImpl.java
@@ -19,91 +19,153 @@
 package org.apache.paimon.globalindex;
 
 import org.apache.paimon.Snapshot;
-import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.index.GlobalIndexMeta;
 import org.apache.paimon.index.IndexFileHandler;
+import org.apache.paimon.index.IndexPathFactory;
 import org.apache.paimon.manifest.IndexManifestEntry;
-import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.partition.PartitionPredicate;
+import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.Filter;
 import org.apache.paimon.utils.Range;
 import org.apache.paimon.utils.SnapshotManager;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 /** Implementation of {@link GlobalIndexScanBuilder}. */
 public class GlobalIndexScanBuilderImpl implements GlobalIndexScanBuilder {
 
-    private final FileStoreTable fileStoreTable;
+    private final Options options;
+    private final RowType rowType;
+    private final FileIO fileIO;
+    private final IndexPathFactory indexPathFactory;
     private final SnapshotManager snapshotManager;
-
-    private Long snapshotId;
-    private BinaryRow partition;
-    private Long rowRangeStart;
-    private Long rowRangeEnd;
-
-    public GlobalIndexScanBuilderImpl(FileStoreTable fileStoreTable) {
-        this.fileStoreTable = fileStoreTable;
-        this.snapshotManager = fileStoreTable.snapshotManager();
+    private final IndexFileHandler indexFileHandler;
+
+    private Snapshot snapshot;
+    private PartitionPredicate partitionPredicate;
+    private Range rowRange;
+
+    public GlobalIndexScanBuilderImpl(
+            Options options,
+            RowType rowType,
+            FileIO fileIO,
+            IndexPathFactory indexPathFactory,
+            SnapshotManager snapshotManager,
+            IndexFileHandler indexFileHandler) {
+        this.options = options;
+        this.rowType = rowType;
+        this.fileIO = fileIO;
+        this.indexPathFactory = indexPathFactory;
+        this.snapshotManager = snapshotManager;
+        this.indexFileHandler = indexFileHandler;
     }
 
     @Override
     public GlobalIndexScanBuilder withSnapshot(long snapshotId) {
-        this.snapshotId = snapshotId;
+        this.snapshot = snapshotManager.snapshot(snapshotId);
+        return this;
+    }
+
+    @Override
+    public GlobalIndexScanBuilder withSnapshot(Snapshot snapshot) {
+        this.snapshot = snapshot;
         return this;
     }
 
     @Override
-    public GlobalIndexScanBuilder withPartition(BinaryRow binaryRow) {
-        this.partition = binaryRow;
+    public GlobalIndexScanBuilder withPartitionPredicate(PartitionPredicate 
partitionPredicate) {
+        this.partitionPredicate = partitionPredicate;
         return this;
     }
 
     @Override
     public GlobalIndexScanBuilder withRowRange(Range rowRange) {
-        this.rowRangeStart = rowRange.from;
-        this.rowRangeEnd = rowRange.to;
+        this.rowRange = rowRange;
         return this;
     }
 
     @Override
     public RowRangeGlobalIndexScanner build() {
-        Objects.requireNonNull(rowRangeStart, "rowRangeStart must not be 
null");
-        Objects.requireNonNull(rowRangeEnd, "rowRangeEnd must not be null");
+        Objects.requireNonNull(rowRange, "rowRange must not be null");
         List<IndexManifestEntry> entries = scan();
-        return new RowRangeGlobalIndexScanner(fileStoreTable, rowRangeStart, 
rowRangeEnd, entries);
+        return new RowRangeGlobalIndexScanner(
+                options, rowType, fileIO, indexPathFactory, rowRange, entries);
     }
 
     @Override
-    public Set<Range> shardList() {
-        return scan().stream()
-                .map(
-                        entry -> {
-                            GlobalIndexMeta globalIndexMeta = 
entry.indexFile().globalIndexMeta();
-                            if (globalIndexMeta == null) {
-                                return null;
-                            }
-                            long start = globalIndexMeta.rowRangeStart();
-                            long end = globalIndexMeta.rowRangeEnd();
-                            return new Range(start, end);
-                        })
-                .filter(Objects::nonNull)
-                .collect(Collectors.toSet());
+    public List<Range> shardList() {
+
+        Map<String, List<Range>> indexRanges = new HashMap<>();
+        for (IndexManifestEntry entry : scan()) {
+            GlobalIndexMeta globalIndexMeta = 
entry.indexFile().globalIndexMeta();
+
+            if (globalIndexMeta == null) {
+                continue;
+            }
+            long start = globalIndexMeta.rowRangeStart();
+            long end = globalIndexMeta.rowRangeEnd();
+            indexRanges
+                    .computeIfAbsent(entry.indexFile().indexType(), k -> new 
ArrayList<>())
+                    .add(new Range(start, end));
+        }
+
+        String checkIndexType = null;
+        List<Range> checkRanges = null;
+        // check all type index have same shard ranges
+        // If index a has [1,10],[20,30] and index b has [1,10],[20,25], it's 
inconsistent, because
+        // it is hard to handle the [26,30] range.
+        for (Map.Entry<String, List<Range>> rangeEntry : 
indexRanges.entrySet()) {
+            String indexType = rangeEntry.getKey();
+            List<Range> ranges = rangeEntry.getValue();
+            if (checkRanges == null) {
+                checkIndexType = indexType;
+                checkRanges = Range.sortAndMergeOverlap(ranges, true);
+            } else {
+                List<Range> merged = Range.sortAndMergeOverlap(ranges, true);
+                if (merged.size() != checkRanges.size()) {
+                    throw new IllegalStateException(
+                            "Inconsistent shard ranges among index types: "
+                                    + checkIndexType
+                                    + " vs "
+                                    + indexType);
+                }
+                for (int i = 0; i < merged.size(); i++) {
+                    Range r1 = merged.get(i);
+                    Range r2 = checkRanges.get(i);
+                    if (r1.from != r2.from || r1.to != r2.to) {
+                        throw new IllegalStateException(
+                                "Inconsistent shard ranges among index types:"
+                                        + checkIndexType
+                                        + " vs "
+                                        + indexType);
+                    }
+                }
+            }
+        }
+
+        return Range.sortAndMergeOverlap(
+                indexRanges.values().stream()
+                        .flatMap(Collection::stream)
+                        .collect(Collectors.toList()));
     }
 
     private List<IndexManifestEntry> scan() {
-        IndexFileHandler indexFileHandler = 
fileStoreTable.store().newIndexFileHandler();
-
         Filter<IndexManifestEntry> filter =
                 entry -> {
-                    if (partition != null) {
-                        if (!entry.partition().equals(partition)) {
+                    if (partitionPredicate != null) {
+                        if (!partitionPredicate.test(entry.partition())) {
                             return false;
                         }
                     }
-                    if (rowRangeStart != null && rowRangeEnd != null) {
+                    if (rowRange != null) {
                         GlobalIndexMeta globalIndexMeta = 
entry.indexFile().globalIndexMeta();
                         if (globalIndexMeta == null) {
                             return false;
@@ -111,7 +173,7 @@ public class GlobalIndexScanBuilderImpl implements 
GlobalIndexScanBuilder {
                         long entryStart = globalIndexMeta.rowRangeStart();
                         long entryEnd = globalIndexMeta.rowRangeEnd();
 
-                        if (!Range.intersect(entryStart, entryEnd, 
rowRangeStart, rowRangeEnd)) {
+                        if (!Range.intersect(entryStart, entryEnd, 
rowRange.from, rowRange.to)) {
                             return false;
                         }
                     }
@@ -119,9 +181,7 @@ public class GlobalIndexScanBuilderImpl implements 
GlobalIndexScanBuilder {
                 };
 
         Snapshot snapshot =
-                snapshotId == null
-                        ? snapshotManager.latestSnapshot()
-                        : snapshotManager.snapshot(snapshotId);
+                this.snapshot == null ? snapshotManager.latestSnapshot() : 
this.snapshot;
 
         return indexFileHandler.scan(snapshot, filter);
     }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/IndexedSplit.java 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/IndexedSplit.java
index c0d4b7a363..2f40137fd8 100644
--- a/paimon-core/src/main/java/org/apache/paimon/globalindex/IndexedSplit.java
+++ b/paimon-core/src/main/java/org/apache/paimon/globalindex/IndexedSplit.java
@@ -24,6 +24,7 @@ import org.apache.paimon.io.DataOutputView;
 import org.apache.paimon.io.DataOutputViewStreamWrapper;
 import org.apache.paimon.table.source.DataSplit;
 import org.apache.paimon.table.source.Split;
+import org.apache.paimon.utils.FloatUtils;
 import org.apache.paimon.utils.Range;
 
 import javax.annotation.Nullable;
@@ -83,7 +84,7 @@ public class IndexedSplit implements Split {
 
         return split.equals(that.split)
                 && rowRanges.equals(that.rowRanges)
-                && Arrays.equals(scores, that.scores);
+                && FloatUtils.equals(scores, that.scores, 0.00001f);
     }
 
     @Override
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/RowRangeGlobalIndexScanner.java
 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/RowRangeGlobalIndexScanner.java
index 94ec64e20c..07bc422ad6 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/globalindex/RowRangeGlobalIndexScanner.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/globalindex/RowRangeGlobalIndexScanner.java
@@ -21,11 +21,11 @@ package org.apache.paimon.globalindex;
 import org.apache.paimon.fs.FileIO;
 import org.apache.paimon.index.GlobalIndexMeta;
 import org.apache.paimon.index.IndexFileMeta;
+import org.apache.paimon.index.IndexPathFactory;
 import org.apache.paimon.manifest.IndexManifestEntry;
 import org.apache.paimon.options.Options;
 import org.apache.paimon.predicate.Predicate;
-import org.apache.paimon.table.FileStoreTable;
-import org.apache.paimon.types.DataType;
+import org.apache.paimon.types.DataField;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.Range;
 
@@ -53,31 +53,28 @@ public class RowRangeGlobalIndexScanner implements 
Closeable {
     private final GlobalIndexEvaluator globalIndexEvaluator;
 
     public RowRangeGlobalIndexScanner(
-            FileStoreTable fileStoreTable,
-            long rowRangeStart,
-            long rowRangeEnd,
+            Options options,
+            RowType rowType,
+            FileIO fileIO,
+            IndexPathFactory indexPathFactory,
+            Range range,
             List<IndexManifestEntry> entries) {
-        this.options = fileStoreTable.coreOptions().toConfiguration();
+        this.options = options;
         for (IndexManifestEntry entry : entries) {
             GlobalIndexMeta meta = entry.indexFile().globalIndexMeta();
             checkArgument(
                     meta != null
                             && Range.intersect(
-                                    rowRangeStart,
-                                    rowRangeEnd,
-                                    meta.rowRangeStart(),
-                                    meta.rowRangeEnd()),
+                                    range.from, range.to, 
meta.rowRangeStart(), meta.rowRangeEnd()),
                     "All index files must have an intersection with row range 
["
-                            + rowRangeStart
+                            + range.from
                             + ", "
-                            + rowRangeEnd
+                            + range.to
                             + ")");
         }
 
-        FileIO fileIO = fileStoreTable.fileIO();
         GlobalIndexFileReadWrite indexFileReadWrite =
-                new GlobalIndexFileReadWrite(
-                        fileIO, 
fileStoreTable.store().pathFactory().globalIndexFileFactory());
+                new GlobalIndexFileReadWrite(fileIO, indexPathFactory);
 
         Map<Integer, Map<String, List<IndexFileMeta>>> indexMetas = new 
HashMap<>();
         for (IndexManifestEntry entry : entries) {
@@ -91,14 +88,12 @@ public class RowRangeGlobalIndexScanner implements 
Closeable {
                     .add(entry.indexFile());
         }
 
-        RowType rowType = fileStoreTable.rowType();
-
         IntFunction<Collection<GlobalIndexReader>> readersFunction =
                 fieldId ->
                         createReaders(
                                 indexFileReadWrite,
                                 indexMetas.get(fieldId),
-                                rowType.getField(fieldId).type());
+                                rowType.getField(fieldId));
         this.globalIndexEvaluator = new GlobalIndexEvaluator(rowType, 
readersFunction);
     }
 
@@ -109,7 +104,7 @@ public class RowRangeGlobalIndexScanner implements 
Closeable {
     private Collection<GlobalIndexReader> createReaders(
             GlobalIndexFileReadWrite indexFileReadWrite,
             Map<String, List<IndexFileMeta>> indexMetas,
-            DataType fieldType) {
+            DataField dataField) {
         if (indexMetas == null) {
             return Collections.emptyList();
         }
@@ -121,7 +116,7 @@ public class RowRangeGlobalIndexScanner implements 
Closeable {
                 List<IndexFileMeta> metas = entry.getValue();
                 GlobalIndexerFactory globalIndexerFactory =
                         GlobalIndexerFactoryUtils.load(indexType);
-                GlobalIndexer globalIndexer = 
globalIndexerFactory.create(fieldType, options);
+                GlobalIndexer globalIndexer = 
globalIndexerFactory.create(dataField, options);
                 List<GlobalIndexIOMeta> globalMetas =
                         
metas.stream().map(this::toGlobalMeta).collect(Collectors.toList());
                 readers.add(globalIndexer.createReader(indexFileReadWrite, 
globalMetas));
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java
 
b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java
index f57830e211..fa11671dc6 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java
@@ -89,7 +89,6 @@ public class DataEvolutionSplitRead implements 
SplitRead<InternalRow> {
     private final FileStorePathFactory pathFactory;
     private final Map<FormatKey, FormatReaderMapping> formatReaderMappings;
     private final Function<Long, TableSchema> schemaFetcher;
-    @Nullable private List<Range> rowRanges;
     @Nullable private VariantAccessInfo[] variantAccess;
 
     protected RowType readRowType;
@@ -141,12 +140,6 @@ public class DataEvolutionSplitRead implements 
SplitRead<InternalRow> {
         return this;
     }
 
-    @Override
-    public SplitRead<InternalRow> withRowRanges(@Nullable List<Range> 
rowRanges) {
-        this.rowRanges = rowRanges;
-        return this;
-    }
-
     @Override
     public RecordReader<InternalRow> createReader(Split split) throws 
IOException {
         if (split instanceof DataSplit) {
@@ -157,7 +150,7 @@ public class DataEvolutionSplitRead implements 
SplitRead<InternalRow> {
     }
 
     private RecordReader<InternalRow> createReader(DataSplit dataSplit) throws 
IOException {
-        return createReader(dataSplit, this.rowRanges, this.readRowType);
+        return createReader(dataSplit, null, this.readRowType);
     }
 
     private RecordReader<InternalRow> createReader(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java 
b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java
index c6bb13e238..05ee3ba499 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java
@@ -52,7 +52,6 @@ import org.apache.paimon.utils.FileStorePathFactory;
 import org.apache.paimon.utils.FormatReaderMapping;
 import org.apache.paimon.utils.FormatReaderMapping.Builder;
 import org.apache.paimon.utils.IOExceptionSupplier;
-import org.apache.paimon.utils.Range;
 import org.apache.paimon.utils.RoaringBitmap32;
 
 import org.slf4j.Logger;
@@ -87,7 +86,6 @@ public class RawFileSplitRead implements 
SplitRead<InternalRow> {
     @Nullable private List<Predicate> filters;
     @Nullable private TopN topN;
     @Nullable private Integer limit;
-    @Nullable private List<Range> rowRanges;
     @Nullable private VariantAccessInfo[] variantAccess;
 
     public RawFileSplitRead(
@@ -152,12 +150,6 @@ public class RawFileSplitRead implements 
SplitRead<InternalRow> {
         return this;
     }
 
-    @Override
-    public SplitRead<InternalRow> withRowRanges(@Nullable List<Range> 
rowRanges) {
-        this.rowRanges = rowRanges;
-        return this;
-    }
-
     @Override
     public RecordReader<InternalRow> createReader(Split s) throws IOException {
         DataSplit split = (DataSplit) s;
@@ -270,14 +262,6 @@ public class RawFileSplitRead implements 
SplitRead<InternalRow> {
         if (fileIndexResult instanceof BitmapIndexResult) {
             selection = ((BitmapIndexResult) fileIndexResult).get();
         }
-        if (rowRanges != null) {
-            RoaringBitmap32 selectionRowIds = file.toFileSelection(rowRanges);
-            if (selection == null) {
-                selection = selectionRowIds;
-            } else {
-                selection.and(selectionRowIds);
-            }
-        }
 
         FormatReaderContext formatReaderContext =
                 new FormatReaderContext(
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/operation/SplitRead.java 
b/paimon-core/src/main/java/org/apache/paimon/operation/SplitRead.java
index b1639118ce..2ef1c66507 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/SplitRead.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/SplitRead.java
@@ -26,12 +26,10 @@ import org.apache.paimon.reader.RecordReader;
 import org.apache.paimon.table.source.Split;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.IOFunction;
-import org.apache.paimon.utils.Range;
 
 import javax.annotation.Nullable;
 
 import java.io.IOException;
-import java.util.List;
 
 /**
  * Read operation which provides {@link RecordReader} creation.
@@ -58,10 +56,6 @@ public interface SplitRead<T> {
         return this;
     }
 
-    default SplitRead<T> withRowRanges(@Nullable List<Range> rowRanges) {
-        return this;
-    }
-
     /** Create a {@link RecordReader} from split. */
     RecordReader<T> createReader(Split split) throws IOException;
 
@@ -98,12 +92,6 @@ public interface SplitRead<T> {
                 return this;
             }
 
-            @Override
-            public SplitRead<R> withRowRanges(@Nullable List<Range> rowRanges) 
{
-                read.withRowRanges(rowRanges);
-                return this;
-            }
-
             @Override
             public RecordReader<R> createReader(Split split) throws 
IOException {
                 return splitConvert.apply(split);
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStore.java
 
b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStore.java
index 6ced52bbd1..9b1f8692bb 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStore.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStore.java
@@ -23,6 +23,7 @@ import org.apache.paimon.FileStore;
 import org.apache.paimon.Snapshot;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.fs.Path;
+import org.apache.paimon.globalindex.GlobalIndexScanBuilder;
 import org.apache.paimon.index.IndexFileHandler;
 import org.apache.paimon.manifest.IndexManifestFile;
 import org.apache.paimon.manifest.ManifestFile;
@@ -238,4 +239,9 @@ public class PrivilegedFileStore<T> implements FileStore<T> 
{
     public void setSnapshotCache(Cache<Path, Snapshot> cache) {
         wrapped.setSnapshotCache(cache);
     }
+
+    @Override
+    public GlobalIndexScanBuilder newIndexScanBuilder() {
+        return wrapped.newIndexScanBuilder();
+    }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java 
b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java
index 01f03a7612..a543d0fbbc 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java
@@ -24,7 +24,6 @@ import org.apache.paimon.consumer.ConsumerManager;
 import org.apache.paimon.data.InternalRow;
 import org.apache.paimon.fs.Path;
 import org.apache.paimon.globalindex.GlobalIndexScanBuilder;
-import org.apache.paimon.globalindex.GlobalIndexScanBuilderImpl;
 import org.apache.paimon.operation.LocalOrphanFilesClean;
 import org.apache.paimon.options.ExpireConfig;
 import org.apache.paimon.schema.TableSchema;
@@ -134,7 +133,7 @@ public interface FileStoreTable extends DataTable {
 
     /** Returns a new global index scan builder. */
     default GlobalIndexScanBuilder newIndexScanBuilder() {
-        return new GlobalIndexScanBuilderImpl(this);
+        return store().newIndexScanBuilder();
     }
 
     /**
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java
index c09662aa51..bb4a3cce2b 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java
@@ -26,10 +26,8 @@ import 
org.apache.paimon.predicate.PredicateProjectionConverter;
 import org.apache.paimon.reader.RecordReader;
 import org.apache.paimon.schema.TableSchema;
 import org.apache.paimon.types.RowType;
-import org.apache.paimon.utils.Range;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.Optional;
 
 /** A {@link InnerTableRead} for data table. */
@@ -48,8 +46,6 @@ public abstract class AbstractDataTableRead implements 
InnerTableRead {
 
     public abstract void applyVariantAccess(VariantAccessInfo[] variantAccess);
 
-    public abstract void applyRowRanges(List<Range> rowRanges);
-
     public abstract RecordReader<InternalRow> reader(Split split) throws 
IOException;
 
     @Override
@@ -92,12 +88,6 @@ public abstract class AbstractDataTableRead implements 
InnerTableRead {
         return this;
     }
 
-    @Override
-    public InnerTableRead withRowRanges(List<Range> rowRange) {
-        applyRowRanges(rowRange);
-        return this;
-    }
-
     @Override
     public final RecordReader<InternalRow> createReader(Split split) throws 
IOException {
         RecordReader<InternalRow> reader = reader(split);
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java
index d6bf2386dd..c33a3f58f1 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java
@@ -179,6 +179,12 @@ abstract class AbstractDataTableScan implements 
DataTableScan {
         return this;
     }
 
+    @Nullable
+    @Override
+    public PartitionPredicate partitionFilter() {
+        return snapshotReader.manifestsReader().partitionFilter();
+    }
+
     public CoreOptions options() {
         return options;
     }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java 
b/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java
index c4d54d4375..e47ffcc2ba 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/AppendTableRead.java
@@ -29,7 +29,6 @@ import org.apache.paimon.schema.TableSchema;
 import org.apache.paimon.table.source.splitread.SplitReadConfig;
 import org.apache.paimon.table.source.splitread.SplitReadProvider;
 import org.apache.paimon.types.RowType;
-import org.apache.paimon.utils.Range;
 
 import javax.annotation.Nullable;
 
@@ -50,7 +49,6 @@ public final class AppendTableRead extends 
AbstractDataTableRead {
     private Predicate predicate = null;
     private TopN topN = null;
     private Integer limit = null;
-    @Nullable private List<Range> rowRanges;
     @Nullable private VariantAccessInfo[] variantAccess;
 
     public AppendTableRead(
@@ -80,7 +78,6 @@ public final class AppendTableRead extends 
AbstractDataTableRead {
         read.withFilter(predicate);
         read.withTopN(topN);
         read.withLimit(limit);
-        read.withRowRanges(rowRanges);
         read.withVariantAccess(variantAccess);
     }
 
@@ -96,12 +93,6 @@ public final class AppendTableRead extends 
AbstractDataTableRead {
         this.variantAccess = variantAccess;
     }
 
-    @Override
-    public void applyRowRanges(List<Range> rowRanges) {
-        initialized().forEach(r -> r.withRowRanges(rowRanges));
-        this.rowRanges = rowRanges;
-    }
-
     @Override
     protected InnerTableRead innerWithFilter(Predicate predicate) {
         initialized().forEach(r -> r.withFilter(predicate));
@@ -125,10 +116,9 @@ public final class AppendTableRead extends 
AbstractDataTableRead {
 
     @Override
     public RecordReader<InternalRow> reader(Split split) throws IOException {
-        DataSplit dataSplit = (DataSplit) split;
         for (SplitReadProvider readProvider : readProviders) {
-            if (readProvider.match(dataSplit, new 
SplitReadProvider.Context(false))) {
-                return readProvider.get().get().createReader(dataSplit);
+            if (readProvider.match(split, new 
SplitReadProvider.Context(false))) {
+                return readProvider.get().get().createReader(split);
             }
         }
 
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableRead.java 
b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableRead.java
index 431fcf397b..560a95033a 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableRead.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableRead.java
@@ -24,7 +24,6 @@ import org.apache.paimon.predicate.Predicate;
 import org.apache.paimon.predicate.PredicateBuilder;
 import org.apache.paimon.predicate.TopN;
 import org.apache.paimon.types.RowType;
-import org.apache.paimon.utils.Range;
 
 import java.util.List;
 
@@ -78,8 +77,4 @@ public interface InnerTableRead extends TableRead {
     default InnerTableRead withMetricRegistry(MetricRegistry registry) {
         return this;
     }
-
-    default InnerTableRead withRowRanges(List<Range> rowRanges) {
-        return this;
-    }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java 
b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java
index ce05f9c2c3..b4d147d885 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java
@@ -91,4 +91,9 @@ public interface InnerTableScan extends TableScan {
         // do nothing, should implement this if need
         return this;
     }
+
+    @Nullable
+    default PartitionPredicate partitionFilter() {
+        return null;
+    }
 }
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java
 
b/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java
index 0af6e44ea9..2fc597a7cf 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/KeyValueTableRead.java
@@ -36,7 +36,6 @@ import 
org.apache.paimon.table.source.splitread.MergeFileSplitReadProvider;
 import 
org.apache.paimon.table.source.splitread.PrimaryKeyTableRawFileSplitReadProvider;
 import org.apache.paimon.table.source.splitread.SplitReadProvider;
 import org.apache.paimon.types.RowType;
-import org.apache.paimon.utils.Range;
 
 import javax.annotation.Nullable;
 
@@ -116,11 +115,6 @@ public final class KeyValueTableRead extends 
AbstractDataTableRead {
         this.variantAccess = variantAccess;
     }
 
-    @Override
-    public void applyRowRanges(List<Range> rowRanges) {
-        throw new UnsupportedOperationException("Does not support row 
ranges.");
-    }
-
     @Override
     public InnerTableRead forceKeepDelete() {
         initialized().forEach(SplitRead::forceKeepDelete);
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java 
b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java
index a12f3152e8..c40ef86554 100644
--- 
a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java
+++ 
b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java
@@ -21,11 +21,14 @@ package org.apache.paimon.table.source;
 import org.apache.paimon.CoreOptions;
 import org.apache.paimon.data.variant.VariantAccessInfo;
 import org.apache.paimon.data.variant.VariantAccessInfoUtils;
+import org.apache.paimon.globalindex.GlobalIndexBatchScan;
 import org.apache.paimon.partition.PartitionPredicate;
 import org.apache.paimon.predicate.Predicate;
 import org.apache.paimon.predicate.PredicateBuilder;
 import org.apache.paimon.predicate.TopN;
+import org.apache.paimon.table.FileStoreTable;
 import org.apache.paimon.table.InnerTable;
+import org.apache.paimon.table.Table;
 import org.apache.paimon.types.RowType;
 import org.apache.paimon.utils.Filter;
 import org.apache.paimon.utils.Range;
@@ -183,7 +186,11 @@ public class ReadBuilderImpl implements ReadBuilder {
 
     @Override
     public TableScan newScan() {
-        InnerTableScan tableScan = configureScan(table.newScan());
+        InnerTableScan tableScan = table.newScan();
+        if (searchGlobalIndex(table)) {
+            tableScan = new GlobalIndexBatchScan((FileStoreTable) table, 
tableScan);
+        }
+        tableScan = configureScan(tableScan);
         if (limit != null) {
             tableScan.withLimit(limit);
         }
@@ -206,6 +213,7 @@ public class ReadBuilderImpl implements ReadBuilder {
                 .withReadType(readType)
                 .withPartitionFilter(partitionFilter)
                 .withRowRanges(rowRanges);
+
         checkState(
                 bucketFilter == null || shardIndexOfThisSubtask == null,
                 "Bucket filter and shard configuration cannot be used 
together. "
@@ -231,6 +239,13 @@ public class ReadBuilderImpl implements ReadBuilder {
         return scan;
     }
 
+    private boolean searchGlobalIndex(Table table) {
+        return table instanceof FileStoreTable
+                && ((FileStoreTable) 
table).coreOptions().dataEvolutionEnabled()
+                && (((FileStoreTable) table).coreOptions().globalIndexEnabled()
+                        || this.rowRanges != null);
+    }
+
     @Override
     public TableRead newRead() {
         InnerTableRead read = table.newRead().withFilter(filter);
@@ -243,9 +258,6 @@ public class ReadBuilderImpl implements ReadBuilder {
         if (limit != null) {
             read.withLimit(limit);
         }
-        if (rowRanges != null) {
-            read.withRowRanges(rowRanges);
-        }
         if (variantAccessInfo != null) {
             read.withVariantAccess(variantAccessInfo);
         }
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/table/DataEvolutionTableTest.java 
b/paimon-core/src/test/java/org/apache/paimon/table/DataEvolutionTableTest.java
index c7215ce03b..de941ed6ff 100644
--- 
a/paimon-core/src/test/java/org/apache/paimon/table/DataEvolutionTableTest.java
+++ 
b/paimon-core/src/test/java/org/apache/paimon/table/DataEvolutionTableTest.java
@@ -31,6 +31,7 @@ import org.apache.paimon.globalindex.GlobalIndexWriter;
 import org.apache.paimon.globalindex.GlobalIndexer;
 import org.apache.paimon.globalindex.GlobalIndexerFactory;
 import org.apache.paimon.globalindex.GlobalIndexerFactoryUtils;
+import org.apache.paimon.globalindex.IndexedSplit;
 import org.apache.paimon.globalindex.RowRangeGlobalIndexScanner;
 import org.apache.paimon.globalindex.bitmap.BitmapGlobalIndexerFactory;
 import org.apache.paimon.index.GlobalIndexMeta;
@@ -67,7 +68,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
-import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
@@ -554,7 +554,7 @@ public class DataEvolutionTableTest extends TableTestBase {
                 .isEqualTo(1); // Should return one split containing the first 
file
 
         // Verify the split contains only the first file (firstRowId=0, 
rowCount=2)
-        DataSplit dataSplit1 = (DataSplit) splits1.get(0);
+        DataSplit dataSplit1 = ((IndexedSplit) splits1.get(0)).dataSplit();
         assertThat(dataSplit1.dataFiles().size()).isEqualTo(1);
         DataFileMeta file1 = dataSplit1.dataFiles().get(0);
         assertThat(file1.firstRowId()).isEqualTo(0L);
@@ -567,7 +567,7 @@ public class DataEvolutionTableTest extends TableTestBase {
                 .isEqualTo(1); // Should return one split containing the 
second file
 
         // Verify the split contains only the second file (firstRowId=2, 
rowCount=2)
-        DataSplit dataSplit2 = (DataSplit) splits2.get(0);
+        DataSplit dataSplit2 = ((IndexedSplit) splits2.get(0)).dataSplit();
         assertThat(dataSplit2.dataFiles().size()).isEqualTo(1);
         DataFileMeta file2 = dataSplit2.dataFiles().get(0);
         assertThat(file2.firstRowId()).isEqualTo(2L);
@@ -580,7 +580,7 @@ public class DataEvolutionTableTest extends TableTestBase {
                 .isEqualTo(1); // Should return one split containing the third 
file
 
         // Verify the split contains only the third file (firstRowId=4, 
rowCount=2)
-        DataSplit dataSplit3 = (DataSplit) splits3.get(0);
+        DataSplit dataSplit3 = ((IndexedSplit) splits3.get(0)).dataSplit();
         assertThat(dataSplit3.dataFiles().size()).isEqualTo(1);
         DataFileMeta file3 = dataSplit3.dataFiles().get(0);
         assertThat(file3.firstRowId()).isEqualTo(4L);
@@ -593,7 +593,7 @@ public class DataEvolutionTableTest extends TableTestBase {
                 .isEqualTo(1); // Should return one split containing all 
matching files
 
         // Verify the split contains all three files (firstRowId=0,2,4)
-        DataSplit dataSplit4 = (DataSplit) splits4.get(0);
+        DataSplit dataSplit4 = ((IndexedSplit) splits4.get(0)).dataSplit();
         assertThat(dataSplit4.dataFiles().size()).isEqualTo(2);
 
         // Check that all three files are present with correct firstRowIds
@@ -654,7 +654,7 @@ public class DataEvolutionTableTest extends TableTestBase {
                 .isEqualTo(1); // Should return one split containing the first 
file
 
         // Verify the split contains only the first file (firstRowId=0)
-        DataSplit dataSplit8 = (DataSplit) splits8.get(0);
+        DataSplit dataSplit8 = ((IndexedSplit) splits8.get(0)).dataSplit();
         assertThat(dataSplit8.dataFiles().size()).isEqualTo(1);
         DataFileMeta file8 = dataSplit8.dataFiles().get(0);
         assertThat(file8.firstRowId()).isEqualTo(0L);
@@ -691,7 +691,7 @@ public class DataEvolutionTableTest extends TableTestBase {
         List<Split> split10 = 
readBuilder.withRowRanges(rowIds10).newScan().plan().splits();
 
         // without projection, all datafiles needed to assemble a row should 
be scanned out
-        List<DataFileMeta> fileMetas10 = ((DataSplit) 
split10.get(0)).dataFiles();
+        List<DataFileMeta> fileMetas10 = (((IndexedSplit) 
split10.get(0)).dataSplit()).dataFiles();
         assertThat(fileMetas10.size()).isEqualTo(2);
 
         List<Range> rowIds11 = Collections.singletonList(new Range(0L, 0L));
@@ -704,7 +704,7 @@ public class DataEvolutionTableTest extends TableTestBase {
                         .splits();
 
         // with projection, irrelevant datafiles should be filtered
-        List<DataFileMeta> fileMetas11 = ((DataSplit) 
split11.get(0)).dataFiles();
+        List<DataFileMeta> fileMetas11 = (((IndexedSplit) 
split11.get(0)).dataSplit()).dataFiles();
         assertThat(fileMetas11.size()).isEqualTo(1);
     }
 
@@ -771,7 +771,7 @@ public class DataEvolutionTableTest extends TableTestBase {
         List<Range> rowIds = Arrays.asList(new Range(0L, 0L), new Range(3L, 
3L));
         List<Split> splits = 
readBuilder.withRowRanges(rowIds).newScan().plan().splits();
         assertThat(splits.size()).isEqualTo(1);
-        DataSplit dataSplit = (DataSplit) splits.get(0);
+        DataSplit dataSplit = ((IndexedSplit) splits.get(0)).dataSplit();
         assertThat(dataSplit.dataFiles().size()).isEqualTo(2);
         DataFileMeta file1 = dataSplit.dataFiles().get(0);
         assertThat(file1.firstRowId()).isEqualTo(0L);
@@ -824,25 +824,98 @@ public class DataEvolutionTableTest extends TableTestBase 
{
 
     @Test
     public void testGlobalIndex() throws Exception {
+        write(100000L);
+
+        FileStoreTable table = (FileStoreTable) catalog.getTable(identifier());
+
+        Predicate predicate =
+                new PredicateBuilder(table.rowType()).equal(1, 
BinaryString.fromString("a100"));
+
+        RoaringNavigableMap64 rowIds = globalIndexScan(table, predicate);
+        assertNotNull(rowIds);
+        Assertions.assertThat(rowIds.getLongCardinality()).isEqualTo(1);
+        Assertions.assertThat(rowIds.toRangeList()).containsExactly(new 
Range(100L, 100L));
+
+        Predicate predicate2 =
+                new PredicateBuilder(table.rowType())
+                        .in(
+                                1,
+                                Arrays.asList(
+                                        BinaryString.fromString("a200"),
+                                        BinaryString.fromString("a300"),
+                                        BinaryString.fromString("a400")));
+
+        rowIds = globalIndexScan(table, predicate2);
+        assertNotNull(rowIds);
+        Assertions.assertThat(rowIds.getLongCardinality()).isEqualTo(3);
+        Assertions.assertThat(rowIds.toRangeList())
+                .containsExactlyInAnyOrder(
+                        new Range(200L, 200L), new Range(300L, 300L), new 
Range(400L, 400L));
+
+        ReadBuilder readBuilder = 
table.newReadBuilder().withRowRanges(rowIds.toRangeList());
+
+        List<String> readF1 = new ArrayList<>();
+        readBuilder
+                .newRead()
+                .createReader(readBuilder.newScan().plan())
+                .forEachRemaining(
+                        row -> {
+                            readF1.add(row.getString(1).toString());
+                        });
+
+        Assertions.assertThat(readF1).containsExactly("a200", "a300", "a400");
+    }
+
+    @Test
+    public void testGlobalIndexWithCoreScan() throws Exception {
+        write(100000L);
+        FileStoreTable table = (FileStoreTable) catalog.getTable(identifier());
+
+        Predicate predicate =
+                new PredicateBuilder(table.rowType())
+                        .in(
+                                1,
+                                Arrays.asList(
+                                        BinaryString.fromString("a200"),
+                                        BinaryString.fromString("a300"),
+                                        BinaryString.fromString("a400")));
+
+        ReadBuilder readBuilder = table.newReadBuilder().withFilter(predicate);
+
+        List<String> readF1 = new ArrayList<>();
+        readBuilder
+                .newRead()
+                .createReader(readBuilder.newScan().plan())
+                .forEachRemaining(
+                        row -> {
+                            readF1.add(row.getString(1).toString());
+                        });
+
+        Assertions.assertThat(readF1).containsExactly("a200", "a300", "a400");
+    }
+
+    private void write(long count) throws Exception {
         createTableDefault();
-        long count = 100000;
 
         Schema schema = schemaDefault();
         RowType writeType0 = schema.rowType().project(Arrays.asList("f0", 
"f1"));
         RowType writeType1 = 
schema.rowType().project(Collections.singletonList("f2"));
         BatchWriteBuilder builder = getTableDefault().newBatchWriteBuilder();
-        try (BatchTableWrite write0 = 
builder.newWrite().withWriteType(writeType0);
-                BatchTableWrite write1 = 
builder.newWrite().withWriteType(writeType1)) {
-
+        try (BatchTableWrite write0 = 
builder.newWrite().withWriteType(writeType0)) {
             for (int i = 0; i < count; i++) {
                 write0.write(GenericRow.of(i, BinaryString.fromString("a" + 
i)));
-                write1.write(GenericRow.of(BinaryString.fromString("b" + i)));
             }
+            BatchTableCommit commit = builder.newCommit();
+            commit.commit(write0.prepareCommit());
+        }
 
+        builder = getTableDefault().newBatchWriteBuilder();
+        try (BatchTableWrite write1 = 
builder.newWrite().withWriteType(writeType1)) {
+            for (int i = 0; i < count; i++) {
+                write1.write(GenericRow.of(BinaryString.fromString("b" + i)));
+            }
             BatchTableCommit commit = builder.newCommit();
-            List<CommitMessage> commitables = new ArrayList<>();
-            commitables.addAll(write0.prepareCommit());
-            commitables.addAll(write1.prepareCommit());
+            List<CommitMessage> commitables = write1.prepareCommit();
             setFirstRowId(commitables, 0L);
             commit.commit(commitables);
         }
@@ -866,7 +939,7 @@ public class DataEvolutionTableTest extends TableTestBase {
 
         GlobalIndexerFactory globalIndexerFactory =
                 
GlobalIndexerFactoryUtils.load(BitmapGlobalIndexerFactory.IDENTIFIER);
-        GlobalIndexer globalIndexer = 
globalIndexerFactory.create(indexField.type(), new Options());
+        GlobalIndexer globalIndexer = globalIndexerFactory.create(indexField, 
new Options());
         GlobalIndexWriter globaIndexBuilder = 
globalIndexer.createWriter(indexFileReadWrite);
 
         reader.forEachRemaining(r -> globaIndexBuilder.write(r.getString(0)));
@@ -900,57 +973,12 @@ public class DataEvolutionTableTest extends TableTestBase 
{
                         CompactIncrement.emptyIncrement());
 
         
table.newBatchWriteBuilder().newCommit().commit(Collections.singletonList(commitMessage));
-
-        Predicate predicate =
-                new PredicateBuilder(table.rowType()).equal(1, 
BinaryString.fromString("a100"));
-
-        RoaringNavigableMap64 rowIds = globalIndexScan(table, predicate);
-        assertNotNull(rowIds);
-        Assertions.assertThat(rowIds.getLongCardinality()).isEqualTo(1);
-        Assertions.assertThat(rowIds.toRangeList()).containsExactly(new 
Range(100L, 100L));
-
-        Predicate predicate2 =
-                new PredicateBuilder(table.rowType())
-                        .in(
-                                1,
-                                Arrays.asList(
-                                        BinaryString.fromString("a200"),
-                                        BinaryString.fromString("a300"),
-                                        BinaryString.fromString("a400")));
-
-        rowIds = globalIndexScan(table, predicate2);
-        assertNotNull(rowIds);
-        Assertions.assertThat(rowIds.getLongCardinality()).isEqualTo(3);
-        Assertions.assertThat(rowIds.toRangeList())
-                .containsExactlyInAnyOrder(
-                        new Range(200L, 200L), new Range(300L, 300L), new 
Range(400L, 400L));
-
-        readBuilder = 
table.newReadBuilder().withRowRanges(rowIds.toRangeList());
-
-        List<String> readF1 = new ArrayList<>();
-        readBuilder
-                .newRead()
-                .createReader(readBuilder.newScan().plan())
-                .forEachRemaining(
-                        row -> {
-                            readF1.add(row.getString(1).toString());
-                        });
-
-        Assertions.assertThat(readF1).containsExactly("a200", "a300", "a400");
-
-        Predicate predicate3 =
-                new PredicateBuilder(table.rowType()).notEqual(1, 
BinaryString.fromString("a500"));
-        rowIds = globalIndexScan(table, predicate3);
-        assertNotNull(rowIds);
-        Assertions.assertThat(rowIds.getLongCardinality()).isEqualTo(99999L);
-        Assertions.assertThat(rowIds.toRangeList())
-                .contains(new Range(0L, 499L), new Range(501L, 99999L));
     }
 
     private RoaringNavigableMap64 globalIndexScan(FileStoreTable table, 
Predicate predicate)
             throws Exception {
         GlobalIndexScanBuilder indexScanBuilder = table.newIndexScanBuilder();
-        Set<Range> ranges = indexScanBuilder.shardList();
+        List<Range> ranges = indexScanBuilder.shardList();
         GlobalIndexResult globalFileIndexResult = 
GlobalIndexResult.createEmpty();
         for (Range range : ranges) {
             try (RowRangeGlobalIndexScanner scanner =
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java
index 8942e4afb1..f955418b6d 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java
@@ -32,10 +32,8 @@ import org.apache.paimon.table.source.InnerTableRead;
 import org.apache.paimon.table.source.Split;
 import org.apache.paimon.table.source.TableRead;
 import org.apache.paimon.types.RowType;
-import org.apache.paimon.utils.Range;
 
 import java.io.IOException;
-import java.util.List;
 
 import static org.apache.paimon.table.source.KeyValueTableRead.unwrap;
 
@@ -63,11 +61,6 @@ public class LookupCompactDiffRead extends 
AbstractDataTableRead {
         incrementalDiffRead.withVariantAccess(variantAccess);
     }
 
-    @Override
-    public void applyRowRanges(List<Range> rowRanges) {
-        throw new UnsupportedOperationException("Does not support row 
ranges.");
-    }
-
     @Override
     public RecordReader<InternalRow> reader(Split split) throws IOException {
         DataSplit dataSplit = (DataSplit) split;
diff --git 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/globalindex/GlobalIndexBuilder.java
 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/globalindex/GlobalIndexBuilder.java
index 77c1e45aa3..d8d3d8471a 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/globalindex/GlobalIndexBuilder.java
+++ 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/globalindex/GlobalIndexBuilder.java
@@ -22,6 +22,7 @@ import org.apache.paimon.data.InternalRow;
 import org.apache.paimon.globalindex.GlobalIndexFileReadWrite;
 import org.apache.paimon.globalindex.GlobalIndexWriter;
 import org.apache.paimon.globalindex.GlobalIndexer;
+import org.apache.paimon.globalindex.IndexedSplit;
 import org.apache.paimon.globalindex.bitmap.BitmapGlobalIndexerFactory;
 import org.apache.paimon.index.GlobalIndexMeta;
 import org.apache.paimon.index.IndexFileMeta;
@@ -30,13 +31,11 @@ import org.apache.paimon.io.DataIncrement;
 import org.apache.paimon.reader.RecordReader;
 import org.apache.paimon.table.sink.CommitMessage;
 import org.apache.paimon.table.sink.CommitMessageImpl;
-import org.apache.paimon.table.source.DataSplit;
 import org.apache.paimon.table.source.ReadBuilder;
 import org.apache.paimon.utils.Range;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 /** This is a class who truly build index file and generate index metas. */
@@ -48,11 +47,10 @@ public abstract class GlobalIndexBuilder {
         this.context = context;
     }
 
-    public CommitMessage build(DataSplit dataSplit) throws IOException {
+    public CommitMessage build(IndexedSplit indexedSplit) throws IOException {
         ReadBuilder builder = context.table().newReadBuilder();
-        builder.withRowRanges(Collections.singletonList(context.range()))
-                .withReadType(context.readType());
-        RecordReader<InternalRow> rows = 
builder.newRead().createReader(dataSplit);
+        builder.withReadType(context.readType());
+        RecordReader<InternalRow> rows = 
builder.newRead().createReader(indexedSplit);
         List<GlobalIndexWriter.ResultEntry> resultEntries = 
writePaimonRows(context, rows);
         List<IndexFileMeta> indexFileMetas = convertToIndexMeta(context, 
resultEntries);
         DataIncrement dataIncrement = 
DataIncrement.indexIncrement(indexFileMetas);
@@ -66,7 +64,7 @@ public abstract class GlobalIndexBuilder {
         List<IndexFileMeta> results = new ArrayList<>();
         for (GlobalIndexWriter.ResultEntry entry : entries) {
             String fileName = entry.fileName();
-            Range range = entry.rowRange().addOffset(context.range().from);
+            Range range = entry.rowRange().addOffset(context.startOffset());
             GlobalIndexFileReadWrite readWrite = 
context.globalIndexFileReadWrite();
             long fileSize = readWrite.fileSize(fileName);
             GlobalIndexMeta globalIndexMeta =
@@ -87,8 +85,7 @@ public abstract class GlobalIndexBuilder {
     private static List<GlobalIndexWriter.ResultEntry> writePaimonRows(
             GlobalIndexBuilderContext context, RecordReader<InternalRow> rows) 
throws IOException {
         GlobalIndexer globalIndexer =
-                GlobalIndexer.create(
-                        context.indexType(), context.indexField().type(), 
context.options());
+                GlobalIndexer.create(context.indexType(), 
context.indexField(), context.options());
         GlobalIndexWriter globalIndexWriter =
                 globalIndexer.createWriter(context.globalIndexFileReadWrite());
         InternalRow.FieldGetter getter =
diff --git 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/globalindex/GlobalIndexBuilderContext.java
 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/globalindex/GlobalIndexBuilderContext.java
index 10b7ea6b62..8e4a4d7fab 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/globalindex/GlobalIndexBuilderContext.java
+++ 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/globalindex/GlobalIndexBuilderContext.java
@@ -27,7 +27,6 @@ import org.apache.paimon.options.Options;
 import org.apache.paimon.table.FileStoreTable;
 import org.apache.paimon.types.DataField;
 import org.apache.paimon.types.RowType;
-import org.apache.paimon.utils.Range;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -47,7 +46,7 @@ public class GlobalIndexBuilderContext implements 
Serializable {
     private final RowType readType;
     private final DataField indexField;
     private final String indexType;
-    private final Range rowRange;
+    private final long startOffset;
     private final Options options;
 
     public GlobalIndexBuilderContext(
@@ -56,13 +55,13 @@ public class GlobalIndexBuilderContext implements 
Serializable {
             RowType readType,
             DataField indexField,
             String indexType,
-            Range rowRange,
+            long startOffset,
             Options options) {
         this.table = table;
         this.readType = readType;
         this.indexField = indexField;
         this.indexType = indexType;
-        this.rowRange = rowRange;
+        this.startOffset = startOffset;
         this.options = options;
 
         this.binaryRowSerializer = new 
BinaryRowSerializer(partition.getFieldCount());
@@ -97,8 +96,8 @@ public class GlobalIndexBuilderContext implements 
Serializable {
         return indexType;
     }
 
-    public Range range() {
-        return rowRange;
+    public long startOffset() {
+        return startOffset;
     }
 
     public Options options() {
diff --git 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateGlobalIndexProcedure.java
 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateGlobalIndexProcedure.java
index 965a53fcb7..2a28cdf323 100644
--- 
a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateGlobalIndexProcedure.java
+++ 
b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateGlobalIndexProcedure.java
@@ -20,6 +20,7 @@ package org.apache.paimon.spark.procedure;
 
 import org.apache.paimon.data.BinaryRow;
 import org.apache.paimon.fs.Path;
+import org.apache.paimon.globalindex.IndexedSplit;
 import org.apache.paimon.io.DataFileMeta;
 import org.apache.paimon.manifest.ManifestEntry;
 import org.apache.paimon.options.Options;
@@ -60,6 +61,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -174,7 +176,7 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
                                 "Option 'global-index.row-count-per-shard' 
must be greater than 0.");
 
                         // Step 1: generate splits for each partition&&shard
-                        Map<BinaryRow, Map<Range, DataSplit>> splits =
+                        Map<BinaryRow, List<IndexedSplit>> splits =
                                 split(table, partitionPredicate, rowsPerShard);
 
                         // Step 2: build index by certain index system
@@ -203,7 +205,7 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
 
     private List<CommitMessage> buildIndex(
             FileStoreTable table,
-            Map<BinaryRow, Map<Range, DataSplit>> preparedDS,
+            Map<BinaryRow, List<IndexedSplit>> preparedDS,
             String indexType,
             RowType readType,
             DataField indexField,
@@ -211,13 +213,14 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
             throws IOException {
         JavaSparkContext javaSparkContext = new 
JavaSparkContext(spark().sparkContext());
         List<Pair<GlobalIndexBuilderContext, byte[]>> taskList = new 
ArrayList<>();
-        for (Map.Entry<BinaryRow, Map<Range, DataSplit>> entry : 
preparedDS.entrySet()) {
+        for (Map.Entry<BinaryRow, List<IndexedSplit>> entry : 
preparedDS.entrySet()) {
             BinaryRow partition = entry.getKey();
-            Map<Range, DataSplit> partitions = entry.getValue();
+            List<IndexedSplit> partitions = entry.getValue();
 
-            for (Map.Entry<Range, DataSplit> partitionEntry : 
partitions.entrySet()) {
-                Range startOffset = partitionEntry.getKey();
-                DataSplit partitionDS = partitionEntry.getValue();
+            for (IndexedSplit indexedSplit : partitions) {
+                checkArgument(
+                        indexedSplit.rowRanges().size() == 1,
+                        "Each IndexedSplit should contain exactly one row 
range.");
                 GlobalIndexBuilderContext builderContext =
                         new GlobalIndexBuilderContext(
                                 table,
@@ -225,10 +228,10 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
                                 readType,
                                 indexField,
                                 indexType,
-                                startOffset,
+                                indexedSplit.rowRanges().get(0).from,
                                 options);
 
-                byte[] dsBytes = 
InstantiationUtil.serializeObject(partitionDS);
+                byte[] dsBytes = 
InstantiationUtil.serializeObject(indexedSplit);
                 taskList.add(Pair.of(builderContext, dsBytes));
             }
         }
@@ -242,7 +245,7 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
                                             new CommitMessageSerializer();
                                     GlobalIndexBuilderContext builderContext = 
pair.getLeft();
                                     byte[] dataSplitBytes = pair.getRight();
-                                    DataSplit split =
+                                    IndexedSplit split =
                                             
InstantiationUtil.deserializeObject(
                                                     dataSplitBytes,
                                                     
GlobalIndexBuilder.class.getClassLoader());
@@ -271,7 +274,7 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
         }
     }
 
-    protected Map<BinaryRow, Map<Range, DataSplit>> split(
+    protected Map<BinaryRow, List<IndexedSplit>> split(
             FileStoreTable table, PartitionPredicate partitions, long 
rowsPerShard) {
         FileStorePathFactory pathFactory = table.store().pathFactory();
         // Get all manifest entries from the table scan
@@ -295,11 +298,11 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
      * @param pathFactory path factory for creating bucket paths
      * @return map of partition to shard splits
      */
-    public static Map<BinaryRow, Map<Range, DataSplit>> 
groupFilesIntoShardsByPartition(
+    public static Map<BinaryRow, List<IndexedSplit>> 
groupFilesIntoShardsByPartition(
             Map<BinaryRow, List<ManifestEntry>> entriesByPartition,
             long rowsPerShard,
             BiFunction<BinaryRow, Integer, Path> pathFactory) {
-        Map<BinaryRow, Map<Range, DataSplit>> result = new HashMap<>();
+        Map<BinaryRow, List<IndexedSplit>> result = new HashMap<>();
 
         for (Map.Entry<BinaryRow, List<ManifestEntry>> partitionEntry :
                 entriesByPartition.entrySet()) {
@@ -307,7 +310,7 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
             List<ManifestEntry> partitionEntries = partitionEntry.getValue();
 
             // Group files into shards - a file may belong to multiple shards
-            Map<Long, List<DataFileMeta>> filesByShard = new HashMap<>();
+            Map<Long, List<DataFileMeta>> filesByShard = new LinkedHashMap<>();
 
             for (ManifestEntry entry : partitionEntries) {
                 DataFileMeta file = entry.file();
@@ -332,7 +335,7 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
             }
 
             // Create DataSplit for each shard with exact ranges
-            Map<Range, DataSplit> shardSplits = new HashMap<>();
+            List<IndexedSplit> shardSplits = new ArrayList<>();
             for (Map.Entry<Long, List<DataFileMeta>> shardEntry : 
filesByShard.entrySet()) {
                 long startRowId = shardEntry.getKey();
                 List<DataFileMeta> shardFiles = shardEntry.getValue();
@@ -356,7 +359,8 @@ public class CreateGlobalIndexProcedure extends 
BaseProcedure {
                                 .rawConvertible(false)
                                 .build();
 
-                shardSplits.put(range, dataSplit);
+                shardSplits.add(
+                        new IndexedSplit(dataSplit, 
Collections.singletonList(range), null));
             }
 
             if (!shardSplits.isEmpty()) {
diff --git 
a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/procedure/CreateGlobalIndexProcedureTest.java
 
b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/procedure/CreateGlobalIndexProcedureTest.java
index f455f017a3..a525901753 100644
--- 
a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/procedure/CreateGlobalIndexProcedureTest.java
+++ 
b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/procedure/CreateGlobalIndexProcedureTest.java
@@ -21,6 +21,7 @@ package org.apache.paimon.spark.procedure;
 import org.apache.paimon.data.BinaryRow;
 import org.apache.paimon.data.BinaryRowWriter;
 import org.apache.paimon.fs.Path;
+import org.apache.paimon.globalindex.IndexedSplit;
 import org.apache.paimon.io.DataFileMeta;
 import org.apache.paimon.io.PojoDataFileMeta;
 import org.apache.paimon.manifest.FileKind;
@@ -39,6 +40,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.function.BiFunction;
+import java.util.stream.Collectors;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -61,7 +63,7 @@ public class CreateGlobalIndexProcedureTest {
         entriesByPartition.put(partition, Collections.singletonList(entry));
 
         // Execute
-        Map<BinaryRow, Map<Range, DataSplit>> result =
+        Map<BinaryRow, List<IndexedSplit>> result =
                 CreateGlobalIndexProcedure.groupFilesIntoShardsByPartition(
                         entriesByPartition, 1000L, pathFactory);
 
@@ -69,14 +71,14 @@ public class CreateGlobalIndexProcedureTest {
         assertThat(result).hasSize(1);
         assertThat(result).containsKey(partition);
 
-        Map<Range, DataSplit> shardSplits = result.get(partition);
+        List<IndexedSplit> shardSplits = result.get(partition);
         assertThat(shardSplits).hasSize(1);
 
         // Should be in shard [0, 999]
         Range expectedRange = new Range(0L, 999L);
-        assertThat(shardSplits).containsKey(expectedRange);
+        
assertThat(shardSplits.get(0).rowRanges()).containsExactly(expectedRange);
 
-        DataSplit split = shardSplits.get(expectedRange);
+        DataSplit split = shardSplits.get(0).dataSplit();
         assertThat(split.dataFiles()).hasSize(1);
         assertThat(split.dataFiles().get(0)).isEqualTo(file);
     }
@@ -95,13 +97,13 @@ public class CreateGlobalIndexProcedureTest {
         entriesByPartition.put(partition, Collections.singletonList(entry));
 
         // Execute
-        Map<BinaryRow, Map<Range, DataSplit>> result =
+        Map<BinaryRow, List<IndexedSplit>> result =
                 CreateGlobalIndexProcedure.groupFilesIntoShardsByPartition(
                         entriesByPartition, 1000L, pathFactory);
 
         // Verify
         assertThat(result).hasSize(1);
-        Map<Range, DataSplit> shardSplits = result.get(partition);
+        List<IndexedSplit> shardSplits = result.get(partition);
         assertThat(shardSplits).hasSize(3);
 
         // Verify all three shards contain the file
@@ -109,10 +111,14 @@ public class CreateGlobalIndexProcedureTest {
         Range shard1 = new Range(1000L, 1999L);
         Range shard2 = new Range(2000L, 2999L);
 
-        assertThat(shardSplits).containsKeys(shard0, shard1, shard2);
-        assertThat(shardSplits.get(shard0).dataFiles()).contains(file);
-        assertThat(shardSplits.get(shard1).dataFiles()).contains(file);
-        assertThat(shardSplits.get(shard2).dataFiles()).contains(file);
+        assertThat(
+                        shardSplits.stream()
+                                .flatMap(s -> s.rowRanges().stream())
+                                .collect(Collectors.toList()))
+                .containsExactlyInAnyOrder(shard0, shard1, shard2);
+        assertThat(shardSplits.get(0).dataSplit().dataFiles()).contains(file);
+        assertThat(shardSplits.get(1).dataSplit().dataFiles()).contains(file);
+        assertThat(shardSplits.get(2).dataSplit().dataFiles()).contains(file);
     }
 
     @Test
@@ -135,17 +141,17 @@ public class CreateGlobalIndexProcedureTest {
         entriesByPartition.put(partition, entries);
 
         // Execute
-        Map<BinaryRow, Map<Range, DataSplit>> result =
+        Map<BinaryRow, List<IndexedSplit>> result =
                 CreateGlobalIndexProcedure.groupFilesIntoShardsByPartition(
                         entriesByPartition, 1000L, pathFactory);
 
         // Verify
         assertThat(result).hasSize(1);
-        Map<Range, DataSplit> shardSplits = result.get(partition);
+        List<IndexedSplit> shardSplits = result.get(partition);
         assertThat(shardSplits).hasSize(1);
 
         Range expectedRange = new Range(0L, 999L);
-        DataSplit split = shardSplits.get(expectedRange);
+        DataSplit split = shardSplits.get(0).dataSplit();
         assertThat(split.dataFiles()).hasSize(3);
         assertThat(split.dataFiles()).containsExactlyInAnyOrder(file1, file2, 
file3);
     }
@@ -170,13 +176,13 @@ public class CreateGlobalIndexProcedureTest {
         entriesByPartition.put(partition, entries);
 
         // Execute
-        Map<BinaryRow, Map<Range, DataSplit>> result =
+        Map<BinaryRow, List<IndexedSplit>> result =
                 CreateGlobalIndexProcedure.groupFilesIntoShardsByPartition(
                         entriesByPartition, 1000L, pathFactory);
 
         // Verify
         assertThat(result).hasSize(1);
-        Map<Range, DataSplit> shardSplits = result.get(partition);
+        List<IndexedSplit> shardSplits = result.get(partition);
         assertThat(shardSplits).hasSize(3);
 
         // Verify each shard has the correct file
@@ -184,9 +190,14 @@ public class CreateGlobalIndexProcedureTest {
         Range shard1 = new Range(1000L, 1999L);
         Range shard2 = new Range(2000L, 2999L);
 
-        assertThat(shardSplits.get(shard0).dataFiles()).containsExactly(file1);
-        assertThat(shardSplits.get(shard1).dataFiles()).containsExactly(file2);
-        assertThat(shardSplits.get(shard2).dataFiles()).containsExactly(file3);
+        Map<Range, DataSplit> shardToSplit =
+                shardSplits.stream()
+                        .collect(
+                                Collectors.toMap(
+                                        s -> s.rowRanges().get(0), 
IndexedSplit::dataSplit));
+        assertThat(shardToSplit.get(shard0).dataFiles()).contains(file1);
+        assertThat(shardToSplit.get(shard1).dataFiles()).contains(file2);
+        assertThat(shardToSplit.get(shard2).dataFiles()).contains(file3);
     }
 
     @Test
@@ -206,7 +217,7 @@ public class CreateGlobalIndexProcedureTest {
                 partition2, 
Collections.singletonList(createManifestEntry(partition2, file2)));
 
         // Execute
-        Map<BinaryRow, Map<Range, DataSplit>> result =
+        Map<BinaryRow, List<IndexedSplit>> result =
                 CreateGlobalIndexProcedure.groupFilesIntoShardsByPartition(
                         entriesByPartition, 100L, pathFactory);
 
@@ -215,16 +226,25 @@ public class CreateGlobalIndexProcedureTest {
         assertThat(result).containsKeys(partition1, partition2);
 
         // Verify partition1
-        Map<Range, DataSplit> shardSplits1 = result.get(partition1);
+        List<IndexedSplit> shardSplits1 = result.get(partition1);
+
         assertThat(shardSplits1).hasSize(11);
-        assertThat(shardSplits1).containsKey(new Range(1000, 1099));
-        assertThat(shardSplits1.get(new Range(1000L, 
1099L)).dataFiles()).containsExactly(file1);
+        IndexedSplit split =
+                shardSplits1.stream()
+                        .filter(f -> f.rowRanges().contains(new Range(1000, 
1099)))
+                        .findFirst()
+                        .get();
+        assertThat(split.dataSplit().dataFiles()).containsExactly(file1);
 
         // Verify partition2
-        Map<Range, DataSplit> shardSplits2 = result.get(partition2);
+        List<IndexedSplit> shardSplits2 = result.get(partition2);
         assertThat(shardSplits2).hasSize(11);
-        assertThat(shardSplits1).containsKey(new Range(1000, 1099));
-        assertThat(shardSplits2.get(new Range(1000L, 
1099L)).dataFiles()).containsExactly(file2);
+        split =
+                shardSplits2.stream()
+                        .filter(f -> f.rowRanges().contains(new Range(1000, 
1099)))
+                        .findFirst()
+                        .get();
+        assertThat(split.dataSplit().dataFiles()).containsExactly(file2);
     }
 
     @Test
@@ -240,16 +260,16 @@ public class CreateGlobalIndexProcedureTest {
         entriesByPartition.put(partition, Collections.singletonList(entry));
 
         // Execute
-        Map<BinaryRow, Map<Range, DataSplit>> result =
+        Map<BinaryRow, List<IndexedSplit>> result =
                 CreateGlobalIndexProcedure.groupFilesIntoShardsByPartition(
                         entriesByPartition, 1000L, pathFactory);
 
         // Verify - file ending at row 999 should be in shard [0,999] only
         // File covers rows [0, 999]
         assertThat(result).hasSize(1);
-        Map<Range, DataSplit> shardSplits = result.get(partition);
+        List<IndexedSplit> shardSplits = result.get(partition);
         assertThat(shardSplits).hasSize(1);
-        assertThat(shardSplits).containsKey(new Range(0L, 999L));
+        assertThat(shardSplits.get(0).rowRanges()).containsExactly(new 
Range(0L, 999L));
     }
 
     @Test
@@ -265,17 +285,18 @@ public class CreateGlobalIndexProcedureTest {
         entriesByPartition.put(partition, Collections.singletonList(entry));
 
         // Execute with shard size of 10
-        Map<BinaryRow, Map<Range, DataSplit>> result =
+        Map<BinaryRow, List<IndexedSplit>> result =
                 CreateGlobalIndexProcedure.groupFilesIntoShardsByPartition(
                         entriesByPartition, 10L, pathFactory);
 
         // Verify - file should span 3 shards: [0,9], [10,19], [20,29]
         assertThat(result).hasSize(1);
-        Map<Range, DataSplit> shardSplits = result.get(partition);
+        List<IndexedSplit> shardSplits = result.get(partition);
         assertThat(shardSplits).hasSize(3);
 
-        assertThat(shardSplits)
-                .containsKeys(new Range(0L, 9L), new Range(10L, 19L), new 
Range(20L, 29L));
+        assertThat(shardSplits.stream().flatMap(s -> s.rowRanges().stream()))
+                .containsExactlyInAnyOrder(
+                        new Range(0L, 9L), new Range(10L, 19L), new Range(20L, 
29L));
     }
 
     private BinaryRow createPartition(int i) {

Reply via email to