amit-ranjan-de opened a new issue, #5599: URL: https://github.com/apache/hudi/issues/5599
**_Tips before filing an issue_** - Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)? - Yes - Join the mailing list to engage in conversations and get faster support at [email protected]. - Joined - If you have triaged this as a bug, then file an [issue](https://issues.apache.org/jira/projects/HUDI/issues) directly. - Not sure if this is a bug, as it was hard to reproduce it **Describe the problem you faced** We have some hudi jobs which fail, throwing `FileNotFoundException` while reading the parquet files from S3. We observe this exception to be thrown for files whose names in S3 don't match the names in the latest `.commit` file. The file names in S3 and in the .commit file have matching fileId and instantTime but differ in their **writeToken**. **Details** We've a **MoR** table in hudi in which we are periodically upserting data. Compaction runs after 5 deltacommit(s). It had been running fine, but started to fail recently. From the logs, we see such exception. (full stacktrace is at the bottom) ``` java.io.FileNotFoundException: No such file or directory 's3://XXXX/date=2020/46465be4-73c8-42e7-9905-088e15e0b627-0_23-626-12975_20220429052025969.parquet' ``` Comparing the file names present in the latest commit via `commit showfiles` and in the S3 directory via `fsview latest`, we observe that some file names match while some don't. One such example of the difference in the file names File name from the commit file (**20220429052025969.commit**) ``` hudi> commit showfiles --commit 20220429052025969 date=2020/46465be4-73c8-42e7-9905-088e15e0b627-0_23-626-12975_20220429052025969.parquet ``` File present in S3 ``` hudi> fsview latest --partitionPath date=2020 date=2020/46465be4-73c8-42e7-9905-088e15e0b627-0_23-627-13061_20220429052025969.parquet ``` This example file has different names in S3 and in the .commit file. The names have matching fileId and instantTime but differ in their **writeToken**. **The latest commit** From the .hoodie directory in S3, we see that the last successful commit was at the instant `20220429052025969` ``` aws s3 ls s3://XXXX/wallet-XXXX/.hoodie/ ``` ``` ... 2022-04-29 05:17:55 0 20220429051753856.deltacommit.requested 2022-04-29 05:19:43 25851 20220429051753856.deltacommit.inflight 2022-04-29 05:20:21 52061 20220429051753856.deltacommit 2022-04-29 05:20:31 0 20220429052025969.compaction.inflight 2022-04-29 05:20:31 34606 20220429052025969.compaction.requested 2022-04-29 05:30:14 59691 20220429052025969.commit ``` After this commit, we have a series of rollback as the job continued to fail with `FileNotFoundException` ``` 2022-04-29 06:02:26 1230 20220429060225213.rollback.requested 2022-04-29 06:02:27 0 20220429060225213.rollback.inflight 2022-04-29 06:02:34 1531 20220429060225213.rollback ... ``` hudi> commits show CommitTime | Total Bytes Written | Total Files Added | Total Files Updated | Total Partitions Written | Total Records Written | Total Update Records Written | Total Errors ------------------|----------------------|-------------------|---------------------|--------------------------|-----------------------|------------------------------|------------- 20220429052025969 | 561.7 MB | 0 | 61 | 3 | 18297036 | 12586123 | 0 ... hudi> compactions show all Compaction Instant Time | State | Total FileIds to be Compacted ------------------------|-----------|------------------------------ 20220429052025969 | COMPLETED | 61 ... We tried to run `compaction repair` for the instant `20220429052025969`, but that didn't help ``` Result of Repair Operation : <empty> ``` As we can see from the commits, no cleaner ran after the latest commit at `20220429052025969`. Also, there was no other pending compaction. **Expected behavior** - All the file names in the commit file must be the same as in the S3 directory. - Or, is there any utility to synchronize the file names between the commit file and S3. I'm not sure as what could have caused this issue, so any pointers or configs or insights will be helpful. I'll be happy to share further information. **Environment Description** * Hudi version : 0.10.1 * Spark version : 3.1.2 * Hive version : (not using hive in this pipeline, or hive is not affected in this pipeline) * Hadoop version : 3.3.1 * Storage (HDFS/S3/GCS) : S3 * Running on Docker? (yes/no) : no **Additional context** The table is Merge-on-Read with below properties: Property | Value -----------------------------------------------|------------------------------------------- basePath | s3://xxxx/wallet_db5/wallet-xxxx metaPath | s3://xxxx/wallet-xxxx/.hoodie fileSystem | s3 hoodie.compaction.payload.class | <our custom payload class> hoodie.table.type | MERGE_ON_READ hoodie.table.precombine.field | xxxx hoodie.table.partition.fields | xxxx hoodie.archivelog.folder | archived hoodie.timeline.layout.version | 1 hoodie.table.name | wallet_xxxx hoodie.table.recordkey.fields | id hoodie.datasource.write.hive_style_partitioning| true hoodie.table.keygenerator.class | org.apache.hudi.keygen.SimpleKeyGenerator hoodie.populate.meta.fields | true hoodie.table.base.file.format | PARQUET hoodie.datasource.write.partitionpath.urlencode| false hoodie.table.version | 3 **Stack Trace** ``` WARN TaskSetManager: Lost task 32.0 in stage 11.0 (TID 695) (172.35.116.5 executor 1): org.apache.hudi.exception.HoodieIOException: Failed to read footer for parquet s3://XXXX/date=2020/46465be4-73c8-42e7-9905-088e15e0b627-0_23-626-12975_20220429052025969.parquet at org.apache.hudi.common.util.ParquetUtils.readMetadata(ParquetUtils.java:185) at org.apache.hudi.common.util.ParquetUtils.readFooter(ParquetUtils.java:201) at org.apache.hudi.common.util.BaseFileUtils.readMinMaxRecordKeys(BaseFileUtils.java:109) at org.apache.hudi.io.storage.HoodieParquetReader.readMinMaxRecordKeys(HoodieParquetReader.java:49) at org.apache.hudi.io.HoodieRangeInfoHandle.getMinMaxKeys(HoodieRangeInfoHandle.java:39) at org.apache.hudi.index.bloom.HoodieBloomIndex.lambda$loadInvolvedFiles$4cbadf07$1(HoodieBloomIndex.java:149) at org.apache.spark.api.java.JavaPairRDD$.$anonfun$toScalaFunction$1(JavaPairRDD.scala:1070) at scala.collection.Iterator$$anon$10.next(Iterator.scala:459) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62) at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49) at scala.collection.TraversableOnce.to(TraversableOnce.scala:315) at scala.collection.TraversableOnce.to$(TraversableOnce.scala:313) at scala.collection.AbstractIterator.to(Iterator.scala:1429) at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:307) at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:307) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1429) at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:294) at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:288) at scala.collection.AbstractIterator.toArray(Iterator.scala:1429) at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1030) at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2278) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:131) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500) 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:750) Caused by: java.io.FileNotFoundException: No such file or directory 's3://XXXX/date=2020/46465be4-73c8-42e7-9905-088e15e0b627-0_23-626-12975_20220429052025969.parquet' at com.amazon.ws.emr.hadoop.fs.s3n.S3NativeFileSystem.getFileStatus(S3NativeFileSystem.java:532) at com.amazon.ws.emr.hadoop.fs.EmrFileSystem.getFileStatus(EmrFileSystem.java:694) at org.apache.parquet.hadoop.util.HadoopInputFile.fromPath(HadoopInputFile.java:61) at org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:456) at org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:441) at org.apache.hudi.common.util.ParquetUtils.readMetadata(ParquetUtils.java:183) ... 33 more ``` -- 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]
