Open dyang108 opened 2 years ago
@yihua can you take a look, seems like a metadata related issue?
Update: I got it working on an older version of Hudi 0.10.1, so seems like a regression
@dyang108 : is this happening infrequently? or your pipeline is just stuck. We have unit tests, integration tests for metadata table and we haven't this this issue yet. trying to gauge whats diff in your env or set up. looks like we are just trying to read records from metadata table. nothing fancy.
This happened consistently with the command above every time i ran on Hudi version : 0.12.0 (also tried 0.11.1)
The pipeline failed and exited when I saw this issue.
We have the same stacktrace when running on hudi version 0.11.0, spark 3.2.1, EMR 6.7. We have metadata service enabled and our Spark Streaming Query fails each time. This is a COW table
@nsivabalan What might be a general workaround in that situation to unblock the processing? Of course it depends on the root cause. However will deleting and recreating metadata from hudi-cli help ? One other option might be to disable metadata on the current table and proceed.
got it. did you mean, you are using EMR's spark or oss spark? I understand its EMR cluster.
@kasured to unblock the processing, could you try disabling and deleting the metadata table by setting hoodie.metadata.enable=false
in Hudi configs? This automatically deletes the metadata table after a few commits.
@yihua Yes that helped. However I can assume that the same can be done with hudi-cli as I wrote before. medatada delete and metadata create
@nsivabalan Yes we are using amazon bundle for Spark 3.2.1 which is provided by EMR 6.7
yes, you are right. you can disable via hudi-cli as well.
Since we could not reproduce w/ OSS spark, can you reach out to aws support. CC @umehrot2 @rahil-c : Have you folks seen this issue before. seems like simple read from metadata table is failing w/ EMR spark.
I saw this issue with Spark on mesos (on EC2), not EMR Spark
Hi ,
Is there any resolution for this issue yet or any idea by which release this issue can be fixed ? I am also facing the same issue. My test case is very simple - to reload same file twice
When metadata is enabled Bulk Insert works fine , but Upsert Aborts with "Caused by: java.lang.IllegalStateException: Block has already been inflated" When metedata is disabled ( hoodie.metadata.enable = false ) The Upsert works fine.
My test cases mostly depend on Metadata , so I need it to be enabled. Please let me know if there is any other workaround.
Thank you !
cc @nsivabalan to look into this issue, thanks.
Is a fix for this issue planned to be regressed into 13.0 or a 12.x patch release?
I got the same issue. When I excluded the hudi-common from hudi-aws, it worked successfully
Hit this when using Flink 1.16 and Hudi bdb50ddccc9631317dfb06a06abc38cbd3714ce8 on EKS. Metadata table was disabled enabled.
2023-04-19 23:55:23
org.apache.hudi.exception.HoodieMetadataException: Failed to retrieve files in partition s3a://path-to-data/ from metadata
at org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:152)
at org.apache.hudi.metadata.HoodieMetadataFileSystemView.listPartition(HoodieMetadataFileSystemView.java:69)
at org.apache.hudi.common.table.view.AbstractTableFileSystemView.lambda$ensurePartitionLoadedCorrectly$16(AbstractTableFileSystemView.java:428)
at java.base/java.util.concurrent.ConcurrentHashMap.computeIfAbsent(Unknown Source)
at org.apache.hudi.common.table.view.AbstractTableFileSystemView.ensurePartitionLoadedCorrectly(AbstractTableFileSystemView.java:419)
at org.apache.hudi.common.table.view.AbstractTableFileSystemView.getLatestMergedFileSlicesBeforeOrOn(AbstractTableFileSystemView.java:854)
at org.apache.hudi.common.table.view.PriorityBasedFileSystemView.execute(PriorityBasedFileSystemView.java:104)
at org.apache.hudi.common.table.view.PriorityBasedFileSystemView.getLatestMergedFileSlicesBeforeOrOn(PriorityBasedFileSystemView.java:195)
at org.apache.hudi.sink.partitioner.profile.DeltaWriteProfile.smallFilesProfile(DeltaWriteProfile.java:62)
at org.apache.hudi.sink.partitioner.profile.WriteProfile.getSmallFiles(WriteProfile.java:191)
at org.apache.hudi.sink.partitioner.BucketAssigner.getSmallFileAssign(BucketAssigner.java:179)
at org.apache.hudi.sink.partitioner.BucketAssigner.addInsert(BucketAssigner.java:137)
at org.apache.hudi.sink.partitioner.BucketAssignFunction.getNewRecordLocation(BucketAssignFunction.java:215)
at org.apache.hudi.sink.partitioner.BucketAssignFunction.processRecord(BucketAssignFunction.java:200)
at org.apache.hudi.sink.partitioner.BucketAssignFunction.processElement(BucketAssignFunction.java:162)
at org.apache.flink.streaming.api.operators.KeyedProcessOperator.processElement(KeyedProcessOperator.java:83)
at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:233)
at [org.apache.flink.streaming.runtime.io](http://org.apache.flink.streaming.runtime.io/).AbstractStreamTaskNetworkInput.processElement(AbstractStreamTaskNetworkInput.java:134)
at [org.apache.flink.streaming.runtime.io](http://org.apache.flink.streaming.runtime.io/).AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:105)
at [org.apache.flink.streaming.runtime.io](http://org.apache.flink.streaming.runtime.io/).StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:542)
at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231)
at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:831)
at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:780)
at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:935)
at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:914)
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:728)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:550)
at java.base/java.lang.Thread.run(Unknown Source)
Caused by: org.apache.hudi.exception.HoodieException: Exception when reading log file
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.scanInternalV1(AbstractHoodieLogRecordReader.java:375)
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.scanInternal(AbstractHoodieLogRecordReader.java:222)
at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.performScan(HoodieMergedLogRecordScanner.java:199)
at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:115)
at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:74)
at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner$Builder.build(HoodieMergedLogRecordScanner.java:465)
at org.apache.hudi.metadata.HoodieMetadataLogRecordReader$Builder.build(HoodieMetadataLogRecordReader.java:218)
at org.apache.hudi.metadata.HoodieBackedTableMetadata.getLogRecordScanner(HoodieBackedTableMetadata.java:539)
at org.apache.hudi.metadata.HoodieBackedTableMetadata.openReaders(HoodieBackedTableMetadata.java:440)
at org.apache.hudi.metadata.HoodieBackedTableMetadata.getOrCreateReaders(HoodieBackedTableMetadata.java:425)
at org.apache.hudi.metadata.HoodieBackedTableMetadata.lambda$getRecordsByKeys$3(HoodieBackedTableMetadata.java:239)
at java.base/java.util.HashMap.forEach(Unknown Source)
at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:237)
at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordByKey(HoodieBackedTableMetadata.java:152)
at org.apache.hudi.metadata.BaseTableMetadata.fetchAllFilesInPartition(BaseTableMetadata.java:339)
at org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:150)
... 28 more
Caused by: java.lang.IllegalStateException: Block has already been inflated
at org.apache.hudi.common.util.ValidationUtils.checkState(ValidationUtils.java:76)
at org.apache.hudi.common.table.log.block.HoodieLogBlock.inflate(HoodieLogBlock.java:276)
at org.apache.hudi.common.table.log.block.HoodieLogBlock.inflate(HoodieLogBlock.java:287)
at org.apache.hudi.common.table.log.block.HoodieDataBlock.readRecordsFromBlockPayload(HoodieDataBlock.java:166)
at org.apache.hudi.common.table.log.block.HoodieDataBlock.getRecordIterator(HoodieDataBlock.java:128)
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.getRecordsIterator(AbstractHoodieLogRecordReader.java:807)
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.processDataBlock(AbstractHoodieLogRecordReader.java:630)
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.processQueuedBlocksForInstant(AbstractHoodieLogRecordReader.java:674)
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.scanInternalV1(AbstractHoodieLogRecordReader.java:366
I don't know the detailed logic here, but apparently recursively invoking inflate()
when hitting an IOException
will cause the state check to fail.
@kasured to unblock the processing, could you try disabling and deleting the metadata table by setting
hoodie.metadata.enable=false
in Hudi configs? This automatically deletes the metadata table after a few commits.
right, this seems obviously flawed, it is hiding the actual IO Exception, instead throwing an irrevelant block inflated.
@zinking Can you fire a fix for it.
Hey folks,
This issue: https://gist.github.com/envomp/268bdd35a3b3399db59583c0e159c229#file-cover-logs Seems to be a cover-up to real underlying issue which in our case was: https://gist.github.com/envomp/268bdd35a3b3399db59583c0e159c229#file-actual-logs
Which in turn was caused by TIMELINE_SERVER_BASED marker types being unable when using spark structured streaming. Workaround was to disable metadata table.
@envomp Are you setting fs.s3a.connection.maximum
to a higher value. That might fix the Connection timeout issue.
Hey @ad1happy2go
We have the following s3a configurations:
spark.hadoop.fs.s3a.path.style.access: true
spark.hadoop.fs.s3a.threads.max: 64
spark.hadoop.fs.s3a.connection.maximum: 1024
spark.hadoop.fs.s3a.maxRetries: 64
Also tried setting fs.s3a.connection.maximum to 8096 but the issue persisted.
EDIT:
For a table with smaller volume this is how disabling metadata table affected the app duraton time:
same problem, version 0.14.1, on hdfs .
The issue of Block has already been inflated
is due to a bug that is fixed by #7434.
Describe the problem you faced
Deltastreamer with write output to S3 exits unexpectedly when running in continuous mode.
To Reproduce
Steps to reproduce the behavior: I ran the following:
the /etc/spark/work-dir/ looks like this: aws-java-sdk-bundle-1.12.283.jar hadoop-aws-2.6.5.jar hudi-utilities-bundle_2.11-0.12.0.jar scala-library-2.11.12.jar spark-streaming-kafka-0-10_2.11-2.4.8.jar
Expected behavior
I don't expect there to be issues on compaction here.
Environment Description
Hudi version : 0.12.0 (also tried 0.11.1)
Spark version : 2.4.8
Hive version :
Hadoop version : 2.6.5
Storage (HDFS/S3/GCS..) : S3
Running on Docker? (yes/no) : Yes, docker on Mesos
I'm reading from an Avro kafka topic
Additional context
Add any other context about the problem here.
Reading Avro record from Kafka
Stacktrace