You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/05/16 01:26:16 UTC

[GitHub] [incubator-pinot] Jackie-Jiang opened a new pull request #5400: Do not release the PinotDataBuffer when closing the index

Jackie-Jiang opened a new pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400


   The PinotDataBuffers are tracked and maintained inside SegmentDirectory
   for ImmutableSegment and PinotDataBufferMemoryManager for MutableSegment.
   They are created when loading the indexes and released when closing the
   segment. If the PinotDataBuffer gets released when closing the index, and
   if the buffer manager decide to reuse the buffer, the following read on
   the buffer will cause JVM to crash. This can be triggered in
   SegmentPreProcessor when the same indexes need to be opened twice in two
   different preprocessors.
   
   This PR standardize the behavior of indexes to not release (close) the
   PinotDataBuffer when closing the index. Also, after the index is closed,
   we should prevent reading anything from the PinotDataBuffer because the
   buffer might already be closed and it could cause JVM to crash.


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang merged pull request #5400: Do not release the PinotDataBuffer when closing the index

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400


   


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5400: Do not release the PinotDataBuffer when closing the index

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400#discussion_r426943269



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/segment/store/ColumnIndexDirectoryTestHelper.java
##########
@@ -60,11 +60,11 @@ static PinotDataBuffer getIndexBuffer(ColumnIndexDirectory columnDirectory, Stri
   static void verifyMultipleReads(ColumnIndexDirectory columnDirectory, String column, int numIter)
       throws Exception {
     for (int ii = 0; ii < numIter; ii++) {

Review comment:
       Done lol




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5400: Do not release the PinotDataBuffer when closing the index

Posted by GitBox <gi...@apache.org>.
jackjlli commented on a change in pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400#discussion_r426864186



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/segment/store/ColumnIndexDirectoryTestHelper.java
##########
@@ -60,11 +60,11 @@ static PinotDataBuffer getIndexBuffer(ColumnIndexDirectory columnDirectory, Stri
   static void verifyMultipleReads(ColumnIndexDirectory columnDirectory, String column, int numIter)
       throws Exception {
     for (int ii = 0; ii < numIter; ii++) {

Review comment:
       Might be good to rename the index variable?




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5400: Do not release the PinotDataBuffer when closing the index

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400#discussion_r426993371



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/io/readerwriter/impl/FixedByteSingleColumnSingleValueReaderWriter.java
##########
@@ -176,18 +173,15 @@ private int getBufferId(int row) {
 
   private void addBuffer() {
     LOGGER.info("Allocating {} bytes for: {}", _chunkSizeInBytes, _allocationContext);
+    // NOTE: PinotDataBuffer is tracked in the PinotDataBufferMemoryManager. No need to track it inside the class.
     PinotDataBuffer buffer = _memoryManager.allocate(_chunkSizeInBytes, _allocationContext);
-    _dataBuffers.add(buffer);
+    _writers.add(

Review comment:
       In `MutableSegmentImpl.destroy()` where memory manager is closed as the last step.




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #5400: Do not release the PinotDataBuffer when closing the index

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400#discussion_r426973388



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/io/reader/impl/v1/BaseChunkSingleValueReader.java
##########
@@ -75,7 +67,7 @@ public BaseChunkSingleValueReader(PinotDataBuffer pinotDataBuffer) {
     headerOffset += Integer.BYTES;
 
     int dataHeaderStart = headerOffset;
-    if (_version > 1) {
+    if (version > 1) {

Review comment:
       The version was made into a member variable so that when we add or bump the version it is available to derived classes. Not sure why you removed it.
   
   Recently we made a version bump and it was painful  and re-factoring had to be done. Some background in PR #5285 

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/io/readerwriter/impl/FixedByteSingleColumnSingleValueReaderWriter.java
##########
@@ -176,18 +173,15 @@ private int getBufferId(int row) {
 
   private void addBuffer() {
     LOGGER.info("Allocating {} bytes for: {}", _chunkSizeInBytes, _allocationContext);
+    // NOTE: PinotDataBuffer is tracked in the PinotDataBufferMemoryManager. No need to track it inside the class.
     PinotDataBuffer buffer = _memoryManager.allocate(_chunkSizeInBytes, _allocationContext);
-    _dataBuffers.add(buffer);
+    _writers.add(

Review comment:
       when are these closed?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/io/readerwriter/impl/FixedByteSingleColumnMultiValueReaderWriter.java
##########
@@ -132,47 +129,40 @@ public FixedByteSingleColumnMultiValueReaderWriter(int maxNumberOfMultiValuesPer
     _maxNumberOfMultiValuesPerRow = maxNumberOfMultiValuesPerRow;
     _headerSize = rowCountPerChunk * SIZE_OF_INT * NUM_COLS_IN_HEADER;
     _rowCountPerChunk = rowCountPerChunk;
-    addHeaderBuffers();
+    addHeaderBuffer();
     //at least create space for million entries, which for INT translates into 4mb buffer
     _incrementalCapacity = incrementalCapacity;
-    addDataBuffers(initialCapacity);
+    addDataBuffer(initialCapacity);
     //init(_rowCountPerChunk, _columnSizeInBytes, _maxNumberOfMultiValuesPerRow, initialCapacity, _incrementalCapacity);
   }
 
-  private void addHeaderBuffers() {
+  private void addHeaderBuffer() {
     LOGGER.info("Allocating header buffer of size {} for: {}", _headerSize, _context);
-    _headerBuffer = _memoryManager.allocate(_headerSize, _context);
+    // NOTE: PinotDataBuffer is tracked in the PinotDataBufferMemoryManager. No need to track it inside the class.
+    PinotDataBuffer headerBuffer = _memoryManager.allocate(_headerSize, _context);

Review comment:
       Not sure I understand this. is headerBuffer(s) alllocated here closed anywhere? Same q with databBuffers . The reader and writer classes we construct here do not close these buffers, right?




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on pull request #5400: Do not release the PinotDataBuffer when closing the index

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400#issuecomment-630540590


   > I guess the claim is that memory manager needs to be closed at the close of a segment. It will be good to put that comment in the memory manager interface. If for any reason the memory manager is retained across interfaces, a lot of memory will not be released.
   
   Both segment directory and memory manager are closed when destroying the segment, which is the expected behavior. They are responsible of allocating and releasing the buffers, and might reuse the buffer across different calls.
   
   > The 'reused later' comment needs some explanation. What may be the re-use of those buffers?
   
   In SegmentDirectory, we cache the buffer for each index and return the same buffer if the same buffer is requested twice.


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5400: Do not release the PinotDataBuffer when closing the index

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400#discussion_r426992905



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/io/reader/impl/v1/BaseChunkSingleValueReader.java
##########
@@ -75,7 +67,7 @@ public BaseChunkSingleValueReader(PinotDataBuffer pinotDataBuffer) {
     headerOffset += Integer.BYTES;
 
     int dataHeaderStart = headerOffset;
-    if (_version > 1) {
+    if (version > 1) {

Review comment:
       It is not used anywhere, and IDE will have warning and can auto convert it to local.
   I don't see why this can affect version bump as long as you have the version information inside the data buffer.




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5400: Do not release the PinotDataBuffer when closing the index

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5400:
URL: https://github.com/apache/incubator-pinot/pull/5400#discussion_r426992999



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/io/readerwriter/impl/FixedByteSingleColumnMultiValueReaderWriter.java
##########
@@ -132,47 +129,40 @@ public FixedByteSingleColumnMultiValueReaderWriter(int maxNumberOfMultiValuesPer
     _maxNumberOfMultiValuesPerRow = maxNumberOfMultiValuesPerRow;
     _headerSize = rowCountPerChunk * SIZE_OF_INT * NUM_COLS_IN_HEADER;
     _rowCountPerChunk = rowCountPerChunk;
-    addHeaderBuffers();
+    addHeaderBuffer();
     //at least create space for million entries, which for INT translates into 4mb buffer
     _incrementalCapacity = incrementalCapacity;
-    addDataBuffers(initialCapacity);
+    addDataBuffer(initialCapacity);
     //init(_rowCountPerChunk, _columnSizeInBytes, _maxNumberOfMultiValuesPerRow, initialCapacity, _incrementalCapacity);
   }
 
-  private void addHeaderBuffers() {
+  private void addHeaderBuffer() {
     LOGGER.info("Allocating header buffer of size {} for: {}", _headerSize, _context);
-    _headerBuffer = _memoryManager.allocate(_headerSize, _context);
+    // NOTE: PinotDataBuffer is tracked in the PinotDataBufferMemoryManager. No need to track it inside the class.
+    PinotDataBuffer headerBuffer = _memoryManager.allocate(_headerSize, _context);

Review comment:
       MemoryManager allocates and tracks all the buffers. It releases them when the segment is closed.




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org