Hans-Raintree opened a new issue, #13820:
URL: https://github.com/apache/hudi/issues/13820
**Describe the problem you faced**
Incremental reads are broken with NBCC + Simple Bucket Index with MoR.
**To Reproduce**
Steps to reproduce the behavior:
```
import sys
from pyspark.sql import SparkSession
from pyspark.sql import Row
PATH = "/data/hudi_mor_testing3"
spark = (
SparkSession.builder
.appName("hudi-incremental-repro")
.config("spark.serializer","org.apache.spark.serializer.KryoSerializer")
.config("spark.sql.extensions","org.apache.spark.sql.hudi.HoodieSparkSessionExtension")
.config("spark.sql.catalog.spark_catalog","org.apache.spark.sql.hudi.catalog.HoodieCatalog")
.getOrCreate()
)
spark.sparkContext.setLogLevel("ERROR")
def upsert(rows):
opts = {
"hoodie.table.name":"t_bug",
"hoodie.write.concurrency.mode": "NON_BLOCKING_CONCURRENCY_CONTROL",
"hoodie.table.type":"MERGE_ON_READ",
"hoodie.datasource.write.table.type":"MERGE_ON_READ",
"hoodie.datasource.write.recordkey.field":"id",
"hoodie.datasource.write.precombine.field":"ts",
"hoodie.index.type":"BUCKET",
"hoodie.write.lock.provider":
"org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider",
"hoodie.index.bucket.engine":"SIMPLE",
"hoodie.bucket.index.num.buckets":"1",
"hoodie.metadata.enable":"false",
"hoodie.compact.inline": "true",
"hoodie.compact.inline.trigger.strategy": "NUM_COMMITS",
"hoodie.compact.inline.max.delta.commits": "3",
"hoodie.cleaner.policy":"KEEP_LATEST_COMMITS",
"hoodie.clean.max.commits": "2",
"hoodie.clean.async":"false",
"hoodie.clean.automatic":"true",
"hoodie.cleaner.commits.retained":"2",
"hoodie.keep.max.commits":"4",
"hoodie.keep.min.commits":"2",
"hoodie.clean.failed.writes.policy": "LAZY"
}
df = spark.createDataFrame([Row(**r) for r in rows])
(df.write
.format("hudi")
.option('hoodie.datasource.write.operation', 'upsert')
.options(**opts)
.mode("append")
.save(PATH)
)
for i in range(1, 9):
upsert([{"id": "id-1", "ts": i, "rider": f"r1{i}"}])
snap = spark.read.format("hudi").load(PATH)
incr = (spark.read.format("hudi")
.option("hoodie.datasource.query.type","incremental")
.option("hoodie.datasource.read.begin.instanttime","000")
.load(PATH))
snap.show(truncate=False)
incr.show(truncate=False)
spark.stop()
+-------------------+---------------------+------------------+----------------------+--------------------------------------+----+---+-----+
|_hoodie_commit_time|_hoodie_commit_seqno
|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name
|id |ts |rider|
+-------------------+---------------------+------------------+----------------------+--------------------------------------+----+---+-----+
|20250902104201628 |20250902104201628_0_8|id-1 |
|00000000-0000-0000-0000-000000000000-0|id-1|8 |r18 |
+-------------------+---------------------+------------------+----------------------+--------------------------------------+----+---+-----+
+-------------------+---------------------+------------------+----------------------+-------------------------------------------------------------------------+----+---+-----+
|_hoodie_commit_time|_hoodie_commit_seqno
|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name
|id |ts |rider|
+-------------------+---------------------+------------------+----------------------+-------------------------------------------------------------------------+----+---+-----+
|20250902104145772 |20250902104145772_0_6|id-1 |
|00000000-0000-0000-0000-000000000000-0_0-78-649_20250902104150151.parquet|id-1|6
|r16 |
+-------------------+---------------------+------------------+----------------------+-------------------------------------------------------------------------+----+---+-----+
```
Note how the ts and rider are 6 and r16 in the incremental read vs 8 and r18
in the snapshot read.
Files:
02.09.2025 13:41 434 600
00000000-0000-0000-0000-000000000000-0_0-78-649_20250902104150151.parquet
02.09.2025 13:41 835
.00000000-0000-0000-0000-000000000000-0_20250902104153926.log.1_0-85-748
02.09.2025 13:42 835
.00000000-0000-0000-0000-000000000000-0_20250902104201628.log.1_0-97-855
Timeline Tail:
02.09.2025 13:41 3 524 20250902104145772.deltacommit.inflight
02.09.2025 13:41 0 20250902104145772.deltacommit.requested
02.09.2025 13:41 3 884
20250902104145772_20250902104148653.deltacommit
02.09.2025 13:41 0 20250902104150151.compaction.inflight
02.09.2025 13:41 2 748 20250902104150151.compaction.requested
02.09.2025 13:41 3 887
20250902104150151_20250902104153138.commit
02.09.2025 13:41 3 524 20250902104153926.deltacommit.inflight
02.09.2025 13:41 0 20250902104153926.deltacommit.requested
02.09.2025 13:41 3 884
20250902104153926_20250902104157393.deltacommit
02.09.2025 13:42 3 524 20250902104201628.deltacommit.inflight
02.09.2025 13:42 0 20250902104201628.deltacommit.requested
02.09.2025 13:42 3 884
20250902104201628_20250902104204352.deltacommit
02.09.2025 13:42 2 215 20250902104204815.clean.inflight
02.09.2025 13:42 2 215 20250902104204815.clean.requested
02.09.2025 13:42 2 285 20250902104204815_20250902104207152.clean
Could this be related to the fileId's? They're all
00000000-0000-0000-0000-000000000000%
**Expected behavior**
Incremental read gives ts 8 and rider r18
**Environment Description**
* Hudi version : 1.0.2
* Spark version : 3.5
* Storage (HDFS/S3/GCS..) : HDFS
* Running on Docker? (yes/no) : yes
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]