abcdeian opened a new issue, #13753:
URL: https://github.com/apache/hudi/issues/13753
**Describe the problem you faced**
encounter error when using 1.0.2 flink hudi writer (with
hoodie.write.table.version=6 and enable table auto upgrade) to write data to
old hudi table(table.version=6).
error as below:
```
2025-08-22 15:43:49.636 [pool-9-thread-1] ERROR
org.apache.hudi.sink.StreamWriteOperatorCoordinator - Executor executes action
[commits the instant 20250822154307771] error
java.lang.IllegalArgumentException: File
hdfs://cdp-node01:8020/warehouse/tablespace/external/hive/hudi_zs_pa_pk_0820/.hoodie/timeline/20250822154307771.inflight
does not exist!
at
org.apache.hudi.common.util.ValidationUtils.checkArgument(ValidationUtils.java:42)
at
org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2.transitionStateToComplete(ActiveTimelineV2.java:517)
at
org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2.saveAsComplete(ActiveTimelineV2.java:166)
at
org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:287)
at
org.apache.hudi.client.BaseHoodieWriteClient.commitStats(BaseHoodieWriteClient.java:246)
at
org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:111)
at
org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:74)
at
org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:207)
at
org.apache.hudi.sink.StreamWriteOperatorCoordinator.doCommit(StreamWriteOperatorCoordinator.java:588)
at
org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:564)
at
org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$notifyCheckpointComplete$2(StreamWriteOperatorCoordinator.java:263)
at
org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:130)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
```
search insant `20250822154307771`, can find these flink logs:
```
2025-08-22 15:43:07.973 [pool-9-thread-1] INFO
org.apache.hudi.client.BaseHoodieWriteClient - Generate a new instant time:
20250822154307771 action: commit
2025-08-22 15:43:07.973 [pool-9-thread-1] INFO
o.a.h.common.table.timeline.versioning.v1.ActiveTimelineV1 - Creating a new
instant [==>20250822154307771__commit__REQUESTED]
2025-08-22 15:43:07.977 [pool-9-thread-1] INFO
o.a.h.common.table.timeline.versioning.v1.ActiveTimelineV1 - Create new file
for toInstant
?hdfs://cdp-node01:8020/warehouse/tablespace/external/hive/hudi_zs_pa_pk_0820/.hoodie/20250822154307771.inflight
```
seems the hudi using v1 timeline to create instant and using v2 to complete
instant. does this matther?
**To Reproduce**
We can consistently reproduce this error.
Steps to reproduce the behavior:
1. using spark sql to new an old hudi table(hudi-0.15.0 with table.version=6)
```
CREATE TABLE hudi_zs_pa_pk_0820 (id bigint,name string,ds string)USING hudi
PARTITIONED BY (ds)TBLPROPERTIES ('primaryKey' = 'id');
```
2. we can cat the hoodie.properties and get:
```
#Properties saved on 2025-08-22T06:09:30.592Z
#Fri Aug 22 14:09:30 CST 2025
hoodie.datasource.write.drop.partition.columns=false
hoodie.table.partition.fields=ds
hoodie.table.type=COPY_ON_WRITE
hoodie.archivelog.folder=archived
hoodie.timeline.layout.version=1
hoodie.table.version=6
hoodie.table.recordkey.fields=id
hoodie.datasource.write.partitionpath.urlencode=false
hoodie.database.name=default
hoodie.table.name=hudi_zs_pa_pk_0820
hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator
hoodie.datasource.write.hive_style_partitioning=true
hoodie.table.create.schema={"type"\:"record","name"\:"hudi_zs_pa_pk_0820_record","namespace"\:"hoodie.hudi_zs_pa_pk_0820","fields"\:[{"name"\:"id","type"\:["long","null"]},{"name"\:"name","type"\:["string","null"]},{"name"\:"ds","type"\:["string","null"]}]}
hoodie.table.checksum=1289586214
```
3. run this flink sql job with hudi-1.0.2 flink writer
```
CREATE TEMPORARY TABLE `zs_pa_pk_0820_000` (
`id` BIGINT NOT NULL,
`name` VARCHAR,
`ds` VARCHAR,
PRIMARY KEY (`id`) NOT ENFORCED
) WITH (
'connector' = 'hudi',
'path' =
'hdfs://cdp-node01:8020/warehouse/tablespace/external/hive/hudi_zs_pa_pk_0820',
'table.type' = 'MERGE_ON_READ',
'hive_sync.mode' = 'hms',
'hive_sync.db' = 'default',
'hive_sync.table' = 'hudi_zs_pa_pk_0820',
'hive_sync.metastore.uris' = 'thrift://cdp-node01:9083',
'hoodie.datasource.write.recordkey.field' = 'id',
'hoodie.datasource.write.partitionpath.field' = 'ds',
'hoodie.write.table.version' = '6',
'hoodie.write.auto.upgrade' = 'true',
'hoodie.metadata.enable' = 'true',
'precombine.field' = 'id',
'hive_sync.enabled' = 'true',
'hive_sync.conf.dir' = 'this_parameter_is_removed'
);
CREATE TEMPORARY TABLE `datagen_source` (
`id` BIGINT,
`name` VARCHAR
) WITH (
'connector' = 'datagen',
'rows-per-second' = '1'
);
INSERT INTO `zs_pa_pk_0820_000`
(SELECT `id`
, `name`
, '20250802' AS `ds`
FROM `datagen_source`);
```
4. We found that the table version had been upgraded to 8, but the Flink job
subsequently failed.
The issue can be resolved by restarting the Flink job without setting
hoodie.write.table.version.
After the restart, the new job runs successfully and writes data to the
target table.
the hoodie.properties after running this failed job:
```
#Updated at 2025-08-22T06:27:35.761Z
#Fri Aug 22 14:27:35 CST 2025
hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator
hoodie.table.version=8
hoodie.table.initial.version=6
hoodie.database.name=default
hoodie.datasource.write.hive_style_partitioning=true
hoodie.table.metadata.partitions.inflight=
hoodie.table.keygenerator.type=SIMPLE
hoodie.table.checksum=1289586214
hoodie.table.create.schema={"type"\:"record","name"\:"hudi_zs_pa_pk_0820_record","namespace"\:"hoodie.hudi_zs_pa_pk_0820","fields"\:[{"name"\:"id","type"\:["long","null"]},{"name"\:"name","type"\:["string","null"]},{"name"\:"ds","type"\:["string","null"]}]}
hoodie.archivelog.folder=archived
hoodie.record.merge.strategy.id=eeb8d96f-b1e4-49fd-bbf8-28ac514178e5
hoodie.table.name=hudi_zs_pa_pk_0820
hoodie.compaction.payload.class=org.apache.hudi.common.model.DefaultHoodieRecordPayload
hoodie.compaction.record.merger.strategy=eeb8d96f-b1e4-49fd-bbf8-28ac514178e5
hoodie.table.type=COPY_ON_WRITE
hoodie.datasource.write.partitionpath.urlencode=false
hoodie.datasource.write.drop.partition.columns=false
hoodie.table.metadata.partitions=files
hoodie.timeline.layout.version=2
hoodie.record.merge.mode=EVENT_TIME_ORDERING
hoodie.table.recordkey.fields=id
hoodie.table.partition.fields=ds
hoodie.timeline.path=timeline
```
**Expected behavior**
we expected the hudi flink writer will auto upgrade hudi table to version
8(this actually happened) . and also write the data to target table.
**Environment Description**
* Hudi version : 1.0.2
* Spark version : 2.4
* Flink version : 1.15.3
* Hive version : 3.1.3
* Hadoop version : 3.1.1
* Storage (HDFS/S3/GCS..) : hdfs
* Running on Docker? (yes/no) : no
**Additional context**
Add any other context about the problem here.
**Stacktrace**
```Add the stacktrace of the error.```
--
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]