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 2022/09/09 17:07:13 UTC

[GitHub] [iceberg] Kontinuation opened a new pull request, #5735: Core: Fix memory leaks in vectorized reader for parquet

Kontinuation opened a new pull request, #5735:
URL: https://github.com/apache/iceberg/pull/5735

   This patch fixes native memory leaks in the vectorized reader for parquet, which affects both iceberg-core and iceberg-spark:
   
   1. Arrow `BufferAllocator` will be closed when the vectorized reader was closed, which frees native memory it possesses. The buffer allocator will also check for leaked memory on closing.
   2. We've also fixed memory leaks when reading parquet files containing interleaving plain/dictionary pages. Arrow buffer allocator detected this problem and thrown `IllegalStateException` when running the newly added test without this fix:
   
   ```
   org.apache.iceberg.arrow.vectorized.ArrowReaderTest > testInterleavingPlainAndDictionaryPages FAILED
       java.lang.IllegalStateException: Allocator[ArrowBatchReader] closed with outstanding buffers allocated (6).
       Allocator(ArrowBatchReader) 0/6400/9472/9223372036854775807 (res/actual/peak/limit)
         child allocators: 0
         ledgers: 6
           ledger[1924] allocator: ArrowBatchReader), isOwning: , size: , references: 2, life: 22143564008682936..0, allocatorManager: [, life: ] holds 3 buffers. 
               ArrowBuf[5225], address:140478431376008, length:8
               ArrowBuf[5224], address:140478431375888, length:120
               ArrowBuf[5223], address:140478431375888, length:128
           ledger[1923] allocator: ArrowBatchReader), isOwning: , size: , references: 2, life: 22143564003725139..0, allocatorManager: [, life: ] holds 3 buffers. 
               ArrowBuf[5220], address:140478431392784, length:1024
               ArrowBuf[5222], address:140478431393776, length:32
               ArrowBuf[5221], address:140478431392784, length:992
           ledger[1920] allocator: ArrowBatchReader), isOwning: , size: , references: 2, life: 22143563974533216..0, allocatorManager: [, life: ] holds 3 buffers. 
               ArrowBuf[5215], address:140478431392752, length:32
               ArrowBuf[5214], address:140478431391760, length:992
               ArrowBuf[5213], address:140478431391760, length:1024
           ledger[1922] allocator: ArrowBatchReader), isOwning: , size: , references: 1, life: 22143564003418541..0, allocatorManager: [, life: ] holds 1 buffers. 
               ArrowBuf[5219], address:140478431385616, length:2048
           ledger[1921] allocator: ArrowBatchReader), isOwning: , size: , references: 2, life: 22143563988827959..0, allocatorManager: [, life: ] holds 3 buffers. 
               ArrowBuf[5218], address:140478431375880, length:8
               ArrowBuf[5217], address:140478431375760, length:120
               ArrowBuf[5216], address:140478431375760, length:128
           ledger[1919] allocator: ArrowBatchReader), isOwning: , size: , references: 1, life: 22143563974111982..0, allocatorManager: [, life: ] holds 1 buffers. 
               ArrowBuf[5212], address:140478431383568, length:2048
         reservations: 0
           at org.apache.arrow.memory.BaseAllocator.close(BaseAllocator.java:405)
           at org.apache.iceberg.arrow.vectorized.ArrowBatchReader.close(ArrowBatchReader.java:66)
           at org.apache.iceberg.parquet.VectorizedParquetReader$FileIterator.close(VectorizedParquetReader.java:176)
           at org.apache.iceberg.arrow.vectorized.ArrowReader$VectorizedCombinedScanIterator.hasNext(ArrowReader.java:299)
           at org.apache.iceberg.arrow.vectorized.ArrowReaderTest.testInterleavingPlainAndDictionaryPages(ArrowReaderTest.java:347)
   ```
   
   Notice: this fix may break existing workflows since `IllegalStateException` will be raised when any memory leak problems lurking in the vectorized readers get triggered.
   


-- 
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: issues-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] Kontinuation commented on a diff in pull request #5735: Core/Spark: Fix memory leaks in vectorized reader for parquet

Posted by GitBox <gi...@apache.org>.
Kontinuation commented on code in PR #5735:
URL: https://github.com/apache/iceberg/pull/5735#discussion_r967570968


##########
arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java:
##########
@@ -210,6 +210,9 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) {
   }
 
   private void allocateFieldVector(boolean dictionaryEncodedVector) {
+    if (vec != null) {
+      vec.close();
+    }

Review Comment:
   This is the fix for memory leaks when reading parquet files containing interleaving plain/dictionary pages.
   `VectorizedArrowReader.read` allocates arrow vectors when encoding of pages in currently reading row group changes, it does not close the previously allocated vector before allocating new vector, which causes memory leaks.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] nastra commented on pull request #5735: Core/Spark: Fix memory leaks in vectorized reader for parquet

Posted by GitBox <gi...@apache.org>.
nastra commented on PR #5735:
URL: https://github.com/apache/iceberg/pull/5735#issuecomment-1270431011

   @RussellSpitzer is this something you could potentially review?


-- 
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: issues-unsubscribe@iceberg.apache.org

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