You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/02/07 17:41:45 UTC

[GitHub] [iceberg] pPanda-beta commented on pull request #882: fix: Failed to get status issue because of s3 eventual consistency

pPanda-beta commented on pull request #882:
URL: https://github.com/apache/iceberg/pull/882#issuecomment-774716451


   @sudssf 
   Have you tested it with gcs?
   
   We have some corrupted tables on gcs where some parquet file size in bytes saved in manifest were more than the actual parquet file size on gcs. 
   As a result when we try to load the data on presto (or any other system), it fails with the following error : 
   
   ```
   Caused by: java.io.EOFException: Invalid seek offset: position value (259484) must be between 0 and 235520 for 'gs://<our table location>/data/yyyymmdd=20201031/quarter_hour=2145/00025-206-d545bce0-0445-4146-a149-d2148259c9e1-00005.parquet'
   	at com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel.validatePosition(GoogleCloudStorageReadChannel.java:713)
   	at com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel.position(GoogleCloudStorageReadChannel.java:600)
   	at com.google.cloud.hadoop.fs.gcs.GoogleHadoopFSInputStream.seek(GoogleHadoopFSInputStream.java:199)
   	at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:65)
   	at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:65)
   	at io.prestosql.plugin.hive.util.FSDataInputStreamTail.readTail(FSDataInputStreamTail.java:57)
   	at io.prestosql.plugin.hive.parquet.HdfsParquetDataSource.readTail(HdfsParquetDataSource.java:108)
   ```
   
   We had downloaded the files and found that
   
   - The original file size is 235520 bytes only
   - The manifest avro files has the value of fileSizeInBytes as 259484 (corrupted)
   - We checked couple of other corrupted files as well. For all the corrupted files, the **avro manifest had more than the expected size.** 
    
   ### Unfortunately they consider the manifest avro file as the source of truth, because that is much cheaper than the actual fs stat operation on object store file systems (gcs, s3, ...)
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org