You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "Ethan Guo (Jira)" <ji...@apache.org> on 2022/02/07 07:55:00 UTC

[jira] [Commented] (HUDI-3341) Investigate that metadata table cannot be read for hadoop-aws 2.7.x

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

Ethan Guo commented on HUDI-3341:
---------------------------------

Upon further investigation, it turns out that the older hadoop-aws-2.7.4 has different behavior with FSDataInputStream::seek when the seek position is the file size, e.g., at the end of file.  For newer versions, e.g., aws-java-sdk-bundle-1.12.48 + hadoop-aws-3.3.1, such seek does not cause any issue, while for hadoop-aws-2.7.4, the seek of file size is going to fail, causing EOFException, so thatHoodieLogFileReader::isBlockCorrupted fails for the last block in the log file.  The issue is the same as HUDI-140 for GCS.

> Investigate that metadata table cannot be read for hadoop-aws 2.7.x
> -------------------------------------------------------------------
>
>                 Key: HUDI-3341
>                 URL: https://issues.apache.org/jira/browse/HUDI-3341
>             Project: Apache Hudi
>          Issue Type: Task
>          Components: metadata
>            Reporter: Ethan Guo
>            Assignee: Ethan Guo
>            Priority: Blocker
>             Fix For: 0.11.0
>
>
> Environment: spark 2.4.4 + aws-java-sdk-1.7.4 + hadoop-aws-2.7.4, Hudi 0.11.0-SNAPSHOT, metadata table enabled
> On the write path, the ingestion is successful with metadata table updated.  When trying to read the metadata table for listing, e.g., using hudi-cli, the operation fails with the following exception.
> {code:java}
> Failed to retrieve list of partition from metadata
> org.apache.hudi.exception.HoodieMetadataException: Failed to retrieve list of partition from metadata
>     at org.apache.hudi.metadata.BaseTableMetadata.getAllPartitionPaths(BaseTableMetadata.java:110)
>     at org.apache.hudi.cli.commands.MetadataCommand.listPartitions(MetadataCommand.java:208)
>     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>     at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>     at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>     at java.lang.reflect.Method.invoke(Method.java:498)
>     at org.springframework.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:216)
>     at org.springframework.shell.core.SimpleExecutionStrategy.invoke(SimpleExecutionStrategy.java:68)
>     at org.springframework.shell.core.SimpleExecutionStrategy.execute(SimpleExecutionStrategy.java:59)
>     at org.springframework.shell.core.AbstractShell.executeCommand(AbstractShell.java:134)
>     at org.springframework.shell.core.JLineShell.promptLoop(JLineShell.java:533)
>     at org.springframework.shell.core.JLineShell.run(JLineShell.java:179)
>     at java.lang.Thread.run(Thread.java:748)
> Caused by: org.apache.hudi.exception.HoodieException: Exception when reading log file 
>     at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.scan(AbstractHoodieLogRecordReader.java:334)
>     at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.scan(AbstractHoodieLogRecordReader.java:179)
>     at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.performScan(HoodieMergedLogRecordScanner.java:103)
>     at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader.<init>(HoodieMetadataMergedLogRecordReader.java:71)
>     at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader.<init>(HoodieMetadataMergedLogRecordReader.java:51)
>     at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader$Builder.build(HoodieMetadataMergedLogRecordReader.java:246)
>     at org.apache.hudi.metadata.HoodieBackedTableMetadata.getLogRecordScanner(HoodieBackedTableMetadata.java:376)
>     at org.apache.hudi.metadata.HoodieBackedTableMetadata.lambda$openReadersIfNeeded$4(HoodieBackedTableMetadata.java:292)
>     at java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1660)
>     at org.apache.hudi.metadata.HoodieBackedTableMetadata.openReadersIfNeeded(HoodieBackedTableMetadata.java:282)
>     at org.apache.hudi.metadata.HoodieBackedTableMetadata.lambda$getRecordsByKeys$0(HoodieBackedTableMetadata.java:138)
>     at java.util.HashMap.forEach(HashMap.java:1289)
>     at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:137)
>     at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordByKey(HoodieBackedTableMetadata.java:127)
>     at org.apache.hudi.metadata.BaseTableMetadata.fetchAllPartitionPaths(BaseTableMetadata.java:275)
>     at org.apache.hudi.metadata.BaseTableMetadata.getAllPartitionPaths(BaseTableMetadata.java:108)
>     ... 12 more
> Caused by: org.apache.hudi.exception.HoodieIOException: IOException when reading logblock from log file HoodieLogFile{pathStr='s3a://hudi-testing/metadata_test_table_2/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0', fileLen=-1}
>     at org.apache.hudi.common.table.log.HoodieLogFileReader.next(HoodieLogFileReader.java:375)
>     at org.apache.hudi.common.table.log.HoodieLogFormatReader.next(HoodieLogFormatReader.java:120)
>     at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.scan(AbstractHoodieLogRecordReader.java:211)
>     ... 27 more
> Caused by: java.io.IOException: Attempted read on closed stream.
>     at org.apache.http.conn.EofSensorInputStream.isReadAllowed(EofSensorInputStream.java:109)
>     at org.apache.http.conn.EofSensorInputStream.read(EofSensorInputStream.java:135)
>     at java.io.FilterInputStream.read(FilterInputStream.java:133)
>     at java.io.FilterInputStream.read(FilterInputStream.java:133)
>     at java.io.FilterInputStream.read(FilterInputStream.java:133)
>     at com.amazonaws.util.ContentLengthValidationInputStream.read(ContentLengthValidationInputStream.java:77)
>     at java.io.FilterInputStream.read(FilterInputStream.java:133)
>     at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:160)
>     at java.io.DataInputStream.read(DataInputStream.java:149)
>     at java.io.DataInputStream.readFully(DataInputStream.java:195)
>     at org.apache.hudi.common.table.log.HoodieLogFileReader.scanForNextAvailableBlockOffset(HoodieLogFileReader.java:304)
>     at org.apache.hudi.common.table.log.HoodieLogFileReader.createCorruptBlock(HoodieLogFileReader.java:243)
>     at org.apache.hudi.common.table.log.HoodieLogFileReader.readBlock(HoodieLogFileReader.java:154)
>     at org.apache.hudi.common.table.log.HoodieLogFileReader.next(HoodieLogFileReader.java:373)
>     ... 29 more {code}



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