mickjermsurawong-stripe opened a new pull request #1563:
URL: https://github.com/apache/iceberg/pull/1563


   - This PR closes data iterator when it is exhausted in Spark base data 
reader.
   - **Observation**: Using S3 store, _coalesced_ read sees timeout on http 
pool connections from S3A. Using our internal metrics, we do see constant 
number of pending http pool connections. By contrast, reading directly to the 
underlying S3 data path with parquet read (not iceberg) has zero pending 
connections.
   - **Cause**: Base reader for Spark does not close data iterator when it is 
exhausted
   - **Question**: Why does this not happen more commonly iceberg read 
(non-coaleseced)?
     - In Spark, the data reader will be closed after Spark _task_ is finished 
as it is registered in task completion callback: 
https://git.corp.stripe.com/stripe-private-oss-forks/spark/blob/stripe-2.4.4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala#L43
     - However, if the Spark read task has multiple partitions, we will not 
close the last data iterator for each partition. Only when the whole task is 
complete, the iterators will be released. We have leakage during lifetime of a 
task.
     - In a normal read without coalescing, we have each read task 
corresponding each parquet file part. So when we finish reading a parquet file, 
we finish the task and close connection.
     - Coalesced read reveals this edge case as each task is given more 
partitions to process.
   - **Testing**: I only managed to add unit tests for iteration logic here, 
but internally we have s3 mock which verified that this fix resolves the 
timeout issue.
   - I have checked the other two S3A timeout issues 
https://github.com/apache/iceberg/pull/150 and 
https://github.com/apache/iceberg/pull/1474 and on mailing list, and believe 
that this is a distinct issue. 
   
   cc @rdblue 


----------------------------------------------------------------
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:
[email protected]



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

Reply via email to