dramaticlly commented on issue #4168:
URL: https://github.com/apache/iceberg/issues/4168#issuecomment-1185041892

   Just want to share another incidence of this happening in the our production 
Flink application last week. This time we added extra instrumentation and 
collected more data points, but we still failed to get a reproduce of the issue
   
   ## Setup 
   Iceberg 0.13.0 and Flink 13 with S3 fileIO turned on (default config for MPU 
size and threshold)
   
   ## Symptom
   - Iceberg committed a broken snapshot `3619033567453770968` which track a 
non-exist data file 
`00008-0-a771cfe9-b71d-4e84-b784-e3b270b0ff80-00514.parquet` at  2022-07-10 
06:15:00.666 PST
   - Subsequent read of given partition failed with `FileNotFoundException` 
because process engine cannot find the problematic data files on S3
   
   ## Mitigation
   - Use scripts from iceberg library to "delete" the dataFiles, something like
   ```scala
   val table = getIcebergTable(s"$db_name.$tbl_name")
   val df = spark.sql(s"select * from iceberg.$db_name.$tbl_name.files")
   val dataf = df.filter(df("file_path") === dataFileOfInterest).persist
   
   val origDataFileSize=dataf.select("file_size_in_bytes").map(r => 
r.getLong(0)).collect().head
   val origDataFileRecordCount=dataf.select("record_count").map(r => 
r.getLong(0)).collect().head
   val dt = DataFiles.builder(table.spec)
   .withPath(dataFileOfInterest)
   .withFileSizeInBytes(origDataFileSize)
   .withPartitionPath(origPartitionPath)
   .withRecordCount(origDataFileRecordCount)
   .build()
   
   val t = table.newTransaction
   t.newOverwrite().deleteFile(dt).commit()
   t.commitTransaction();
   ```
   - However some of data consumer are using [Spark incremental 
read](https://iceberg.apache.org/docs/latest/spark-queries/#incremental-read) 
to get list of appended data files in between snapshots and mitigation above 
can only help with read from latest snapshot, so there is no easy way for us to 
fix snapshot `3619033567453770968` but skip processing of this corrupted 
snapshot, which result in some data loss
   
   ## Some investigation
   - We spent some time to collect the requests from AWS side and realized the 
S3 multipart upload get aborted at 2022-07-10 06:07:01.619 PST, or 8 minutes 
before snapshot was committed
   - The multipart upload abortion was caused by transient error in 1 of the 
upload-parts like `software.amazon.awssdk.core.exception.SdkClientException: 
Unable to execute HTTP request: Error writing to server` and all subsequent 
parts upload failed as expected.
   - Initially we are thinking maybe there's some exception gets swallowed so 
caller of 
https://github.com/apache/iceberg/blob/master/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java
 might not realize the problem of writing this parquet data files to S3. But we 
also spent some time in this AWS integration tests 
https://github.com/apache/iceberg/blob/master/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java
 and looks like S3OutputStream behave as expected -> If there's exception in 
threads to upload part, `java.util.concurrent.CompletionException` will bubble 
up as expected on `close()`
   
   ## Questions
   - Correct expectation of using S3OutputStream? Is it safe to assume that 
iceberg is ready to commit if all writers can write its data and metadata files 
to fileIO and writers are close without exception?
   - We are curious about call path on how S3OutputStream is wired to 
IcebergStreamWriter ? (assume this is where commit of list of data files 
happens), there's a lot of complexity in between and hard to identify the exact 
caller of S3OutputStream
   - What else does community suggest to help root cause the actual problem and 
prevent it from happening again?
   
   CC @rdblue @danielcweeks @szehon-ho @jackye1995 @singhpk234 @stevenzwu 


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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to