[ 
https://issues.apache.org/jira/browse/HUDI-2894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17451832#comment-17451832
 ] 

sivabalan narayanan edited comment on HUDI-2894 at 12/1/21, 1:45 PM:
---------------------------------------------------------------------

Updates:

We tried adding a retry logic around the IOException on seek() with HFile and 
issue seems to go away. But in general, the exception happens sporadically and 
at times need more than 1 retry to get the call to succeed. But this happens 
only with integ test suite jobs and we don't see this behavior in deltastreamer 
continuous mode or from spark-shell. We were able to make 30+ commits using 
deltastreamer continuous mode w/o any issues(where in metadata table had 2 
compactions completed and all commits were in good shape). Verified via 
hudi-cli by comparing file listing from FS and from metadata.  

 

 

 


was (Author: shivnarayan):
Updates:

We tried adding a retry logic around the IOException on seek() with HFile and 
issue seems to go away. But in general, the exception happens sporadically and 
at times need more than 1 retry to get the call to succeed. But this happens 
only with integ test suite jobs and we don't see this behavior in deltastreamer 
continuous mode or from spark-shell. We were able to make 30+ commits using 
deltastreamer continuous mode w/o any issues. 

 

 

 

> Metadata table read after compaction fails in S3
> ------------------------------------------------
>
>                 Key: HUDI-2894
>                 URL: https://issues.apache.org/jira/browse/HUDI-2894
>             Project: Apache Hudi
>          Issue Type: Bug
>          Components: Writer Core
>            Reporter: sivabalan narayanan
>            Assignee: Manoj Govindassamy
>            Priority: Blocker
>             Fix For: 0.10.0
>
>
> Once compaction in metadata kicks in, future read fails( hunch is reading 
> from base hfile fails). 
>  
> {code:java}
> 21/11/30 15:35:20 INFO HoodieTableMetaClient: Loading HoodieTableMetaClient 
> from s3a://aws-logs-87995575
> 1789-us-west-1/infra-resources-dev/small/emr/home/hadoop/output
> 21/11/30 15:35:20 ERROR HoodieROTablePathFilter: Error checking path 
> :s3a://aws-logs-879955751789-us-wes
> t-1/infra-resources-dev/small/emr/home/hadoop/output/1970/01/04/135ac18a-db3f-4bc1-b376-960fd85a44c1-0_0
> -326-3529_20211130153211490.parquet, under folder: 
> s3a://aws-logs-879955751789-us-west-1/infra-resources
> -dev/small/emr/home/hadoop/output/1970/01/04
> org.apache.hudi.exception.HoodieMetadataException: Failed to retrieve files 
> in partition s3a://aws-logs-
> 879955751789-us-west-1/infra-resources-dev/small/emr/home/hadoop/output/1970/01/04
>  from metadata
>         at 
> org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:124)
>         at 
> org.apache.hudi.metadata.HoodieMetadataFileSystemView.listPartition(HoodieMetadataFileSystemV
> iew.java:65)
>         at 
> org.apache.hudi.common.table.view.AbstractTableFileSystemView.lambda$ensurePartitionLoadedCor
> rectly$9(AbstractTableFileSystemView.java:290)
>         at 
> java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1660)
>         at 
> org.apache.hudi.common.table.view.AbstractTableFileSystemView.ensurePartitionLoadedCorrectly(
> AbstractTableFileSystemView.java:281)
>         at 
> org.apache.hudi.common.table.view.AbstractTableFileSystemView.getLatestBaseFiles(AbstractTabl
> eFileSystemView.java:449)
>         at 
> org.apache.hudi.hadoop.HoodieROTablePathFilter.accept(HoodieROTablePathFilter.java:194)
>         at 
> org.apache.spark.sql.execution.datasources.PathFilterWrapper.accept(InMemoryFileIndex.scala:1
> 65)
>         at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$listLeafFiles$8(HadoopFSUtils.scala:285)
>         at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$listLeafFiles$8$adapted(HadoopFSUtils.scala:285)
>         at 
> scala.collection.TraversableLike.$anonfun$filterImpl$1(TraversableLike.scala:304)
>         at 
> scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
>         at 
> scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
>         at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198)
>         at 
> scala.collection.TraversableLike.filterImpl(TraversableLike.scala:303)
>         at 
> scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:297)
>         at 
> scala.collection.mutable.ArrayOps$ofRef.filterImpl(ArrayOps.scala:198)
>         at scala.collection.TraversableLike.filter(TraversableLike.scala:395)
>         at scala.collection.TraversableLike.filter$(TraversableLike.scala:395)
>         at scala.collection.mutable.ArrayOps$ofRef.filter(ArrayOps.scala:198)
>         at 
> org.apache.spark.util.HadoopFSUtils$.listLeafFiles(HadoopFSUtils.scala:285)
>         at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$6(HadoopFSUtils.scala:136)
>         at scala.collection.immutable.Stream.map(Stream.scala:418)
>         at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$parallelListLeafFilesInternal$4(HadoopFSUtils.scala:126)
>         at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
>         at 
> org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
>         at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>         at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
>         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:506)
>         at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>         at 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>         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)
> Caused by: org.apache.hudi.exception.HoodieIOException: Error merging records 
> from metadata table for  1 key : 
>         at 
> org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:146)
>         at 
> org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordByKey(HoodieBackedTableMetadata.java:124)
>         at 
> org.apache.hudi.metadata.BaseTableMetadata.fetchAllFilesInPartition(BaseTableMetadata.java:184)
>         at 
> org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:122)
>         ... 39 more
> Caused by: java.io.IOException: Positional read of 4387 bytes failed at 
> offset 0 (returned 1626)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1382)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1591)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1470)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:437)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:259)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:634)
>         at 
> org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:584)
>         at 
> org.apache.hudi.io.storage.HoodieHFileReader.getRecordByKey(HoodieHFileReader.java:252)
>         at 
> org.apache.hudi.io.storage.HoodieFileReader.getRecordByKey(HoodieFileReader.java:49)
>         at 
> org.apache.hudi.metadata.HoodieBackedTableMetadata.readFromBaseAndMergeWithLogRecords(HoodieBackedTableMetadata.java:195)
>         at 
> org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:141)
>         ... 42 more
> 21/11/30 15:35:20 ERROR Executor: Exception in task 6.0 in stage 587.0 (TID 
> 6817)
> org.apache.hudi.exception.HoodieException: Error checking path 
> :s3a://aws-logs-879955751789-us-west-1/infra-resources-dev/small/emr/home/hadoop/output/1970/01/04/135ac18a-db3f-4bc1-b376-960fd85a44c1-0_0-326-3529_20211130153211490.parquet,
>  under folder: 
> s3a://aws-logs-879955751789-us-west-1/infra-resources-dev/small/emr/home/hadoop/output/1970/01/04
>         at 
> org.apache.hudi.hadoop.HoodieROTablePathFilter.accept(HoodieROTablePathFilter.java:230)
>         at 
> org.apache.spark.sql.execution.datasources.PathFilterWrapper.accept(InMemoryFileIndex.scala:165)
>         at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$listLeafFiles$8(HadoopFSUtils.scala:285)
>         at 
> org.apache.spark.util.HadoopFSUtils$.$anonfun$listLeafFiles$8$adapted(HadoopFSUtils.scala:285)
>         at 
> scala.collection.TraversableLike.$anonfun$filterImpl$1(TraversableLike.scala:304)
>         at 
> scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36) 
> {code}
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to