Sorry for duplicate emails if received, for the apache email server complained 
sending failures...

Best regards, Heng Su



> 2022年8月17日 下午7:41,Heng Su <permanent.s...@gmail.com> 写道:
> 
> Hi All,
> 
> I was testing flink sql mor ingestion on hudi 0.11.1 release, now encounter 
> some problem and can not overcome.
> 
> For the case I want to keep historical partition's data not to update to the 
> current partition(partitioned by date), 
> I set 'index.global.enabled'=‘false’, so the expected result should be “every 
> new date partition use new fileIds, compaction and clean should perform in 
> each partition, 
> after passing the end of the clean window, the parquet files left in 
> historical partition should be a group files like 
> {fileid}-{token}-{last_compact_ins_time_of_partition}.parquet”, am I right?
> 
> But in actual practise, I found two unnormal points,
> 1. It seems clean service(async) only clean log files, but left the all 
> basefiles, the clean window I set around 1 day
> 2. For the historical partition, the last logfile at 
> ${last_compact_ins_time_of_partition} was never cleaned(these files end 
> writing at 00:00), but in the afterward compaction perform, I can see only 
> these files compact themself, again and again, without basefile in the 
> slice(when debug I found the basefile is empty in the fileslice).
> 
> There is no error in log, but state size get larger and larger for the reason 
> that invalid logfiles continously participated in compaction(see below).
> 
> Parameters:
> Checkpoint interval 20min
> 
> Table options(flink-1.13.6):
> 
> 'connector' = 'hudi',
> 'path' = 
> 'viewfs://ss-hadoop/project/ads/warehouse/test2.db/hudi_press_search_engine_log_upsert
>  
> <viewfs://ss-hadoop/project/ads/warehouse/test2.db/hudi_press_search_engine_log_upsert>',
> 'table.type' = 'MERGE_ON_READ',
> 'hive_sync.enable' = 'false',
> 'write.operation'='upsert',
> 'write.precombine' = 'true',
> 'write.precombine.field' = 'row_time',
> 'write.bucket_assign.tasks'='24',
> 'write.tasks'='18',
> 'read.streaming.enabled' = 'true',
> 'read.tasks' = '18',
> 'read.utc-timezone'='false',
> 'compaction.schedule.enabled'='true',
> 'compaction.async.enabled'='true',
> 'compaction.delta_seconds'='3600',
> 'compaction.trigger.strategy'='num_commits',
> 'compaction.delta_commits'= '3',
> 'compaction.tasks'='18',
> 'compaction.max_memory'='300',
> 'compaction.target_io'='512000',
> 'write.log.max.size'='1024',
> 'metadata.enabled'='true',
> 'metadata.compaction.delta_commits'='10',
> 'hoodie.datasource.write.hive_style_partitioning'='true',
> 'hoodie.datasource.query.type'='snapshot',
> 'hoodie.filesystem.view.incr.timeline.sync.enable'='true',
> 'hive_sync.enable'='false',
> 'changelog.enabled'='false',
> 'index.type'='FLINK_STATE',
> 'index.global.enabled'='false',
> 'clean.async.enabled'='true',
> 'clean.retain_commits'='18',
> 'archive.max_commits'='72',
> 'archive.min_commits'='24'
> 
> The following snapshot can provide more detail:
> 
> 
> Files left on 08-14, which captured at 08-15 17:00, when the instance in 
> 08-14 has already been archieved
> 
> <Xnip2022-08-17_18-54-40.jpeg>
> 
> 
> <粘贴的图形-1.png>
> 
> Filegroups in 08-14
> <Xnip2022-08-17_18-57-51.jpeg>
> 
> 
> Read completedAndCompactionInstants from archive
> 
> <粘贴的图形-2.png>
> 
> The archive compact content at 20220815000020359, which is the first compact 
> at next day
> 
> <Xnip2022-08-17_19-23-21.jpeg>
> 
> 
> I can't understand why and how this happened, and how to fix it, any help 
> will be appreciated, thanks.
> 
> 
> Best regards, Heng Su
> 

Reply via email to