Hans-Raintree opened a new issue, #13827:
URL: https://github.com/apache/hudi/issues/13827
**Describe the problem you faced**
Incremental reads are broken with Flink NBCC + Simple Bucket Index + MoR.
**To Reproduce**
Flink:
```
SET 'execution.runtime-mode' = 'batch';
SET 'sql-client.execution.result-mode' = 'tableau';
DROP TABLE IF EXISTS t_mor;
CREATE TABLE t_mor (
ts BIGINT,
uuid STRING PRIMARY KEY NOT ENFORCED,
rider STRING,
city STRING
) PARTITIONED BY (city)
WITH (
'connector' = 'hudi',
'path' = 'file:///data/hudi_mor_test',
'table.type' = 'MERGE_ON_READ',
'hoodie.write.concurrency.mode' = 'NON_BLOCKING_CONCURRENCY_CONTROL',
'hoodie.write.lock.provider' =
'org.apache.hudi.client.transaction.lock.InProcessLockProvider',
'record.key.field' = 'uuid',
'precombine.field' = 'ts',
'metadata.enabled' = 'false',
'index.key.field' = 'uuid',
'clean.retain_commits' = '2',
'compaction.delta_commits' = '3',
'hoodie.index.type' = 'BUCKET',
'hoodie.index.bucket.engine' = 'SIMPLE',
'hoodie.bucket.index.num.buckets' = '1',
'hoodie.clean.failed.writes.policy' = 'LAZY'
);
INSERT INTO t_mor VALUES
(1,'id-0001', 'r11', 'sf'),
(2,'id-0002', 'r21', 'nyc'),
(3,'id-0003', 'r31', 'la');
```
Spark:
```
PATH = "/data/hudi_mor_test"
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()
)
snap = spark.read.format("hudi").load(PATH).cache()
snap_cnt = snap.count()
incr = (
spark.read.format("hudi")
.option("hoodie.datasource.query.type", "incremental")
.option("hoodie.datasource.read.begin.instanttime", "000")
.option("hoodie.datasource.read.incr.fallback.fulltablescan.enable",
"false")
.load(PATH)
).cache()
incr_cnt = incr.count()
print(f"SNAPSHOT_COUNT={snap_cnt}")
print(f"INCREMENTAL_COUNT={incr_cnt}")
snap.orderBy("ts").show(100, truncate=False)
incr.orderBy("ts").show(100, truncate=False)
spark.stop()
SNAPSHOT_COUNT=3
INCREMENTAL_COUNT=1
+-------------------+----------------------+------------------+----------------------+--------------------------------------+---+-------+-----+----+
|_hoodie_commit_time|_hoodie_commit_seqno
|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name
|ts |uuid |rider|city|
+-------------------+----------------------+------------------+----------------------+--------------------------------------+---+-------+-----+----+
|20250903072719965 |20250903072719965_0_16|id-0001 |sf
|00000000-0000-0000-0000-000000000000-0|1 |id-0001|r11 |sf |
|20250903072719965 |20250903072719965_0_17|id-0002 |nyc
|00000000-0000-0000-0000-000000000000-0|2 |id-0002|r21 |nyc |
|20250903072719965 |20250903072719965_0_18|id-0003 |la
|00000000-0000-0000-0000-000000000000-0|3 |id-0003|r31 |la |
+-------------------+----------------------+------------------+----------------------+--------------------------------------+---+-------+-----+----+
+-------------------+----------------------+------------------+----------------------+--------------------------------------+---+-------+-----+----+
|_hoodie_commit_time|_hoodie_commit_seqno
|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name
|ts |uuid |rider|city|
+-------------------+----------------------+------------------+----------------------+--------------------------------------+---+-------+-----+----+
|20250903072719965 |20250903072719965_0_16|id-0001 |sf
|00000000-0000-0000-0000-000000000000-0|1 |id-0001|r11 |sf |
+-------------------+----------------------+------------------+----------------------+--------------------------------------+---+-------+-----+----+
It also gives these warnings:
25/09/03 07:27:29 WARN HoodieTableFileSystemView: Partition: la is not
available in store
25/09/03 07:27:29 WARN HoodieTableFileSystemView: Partition: nyc is not
available in store
```
**Expected behavior**
Incremental read shows all 3 inserts.
**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]