You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2020/09/28 14:02:29 UTC

[lucene-solr] branch branch_8x updated: Compute RAM usage ByteBuffersDataOutput on the fly. (#1919)

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 3b59906  Compute RAM usage ByteBuffersDataOutput on the fly. (#1919)
3b59906 is described below

commit 3b599064440f5f41492c3ee4c8e715ea6f320f57
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Mon Sep 28 15:08:08 2020 +0200

    Compute RAM usage ByteBuffersDataOutput on the fly. (#1919)
    
    This helps remove the assumption that all blocks have the same size.
---
 .../apache/lucene/store/ByteBuffersDataOutput.java | 15 ++++----
 .../lucene/store/TestByteBuffersDataOutput.java    | 42 ++++++++++++++++++++++
 2 files changed, 50 insertions(+), 7 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataOutput.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataOutput.java
index e61458d..31e6247 100644
--- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataOutput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataOutput.java
@@ -117,6 +117,9 @@ public final class ByteBuffersDataOutput extends DataOutput implements Accountab
    */
   private final ArrayDeque<ByteBuffer> blocks = new ArrayDeque<>();
 
+  /** Cumulative RAM usage across all blocks. */
+  private long ramBytesUsed;
+
   /**
    * The current-or-next write block.
    */
@@ -401,13 +404,8 @@ public final class ByteBuffersDataOutput extends DataOutput implements Accountab
   public long ramBytesUsed() {
     // Return a rough estimation for allocated blocks. Note that we do not make
     // any special distinction for direct memory buffers.
-    ByteBuffer first = blocks.peek();
-    if (first == null) {
-      return 0L;
-    } else {
-      // All blocks have the same capacity.
-      return (first.capacity() + RamUsageEstimator.NUM_BYTES_OBJECT_REF) * blocks.size();
-    }
+    assert ramBytesUsed == blocks.stream().mapToLong(ByteBuffer::capacity).sum() + blocks.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+    return ramBytesUsed;
   }
 
   /**
@@ -423,6 +421,7 @@ public final class ByteBuffersDataOutput extends DataOutput implements Accountab
       blocks.forEach(blockReuse);
     }
     blocks.clear();
+    ramBytesUsed = 0;
     currentBlock = EMPTY;
   }
 
@@ -456,6 +455,7 @@ public final class ByteBuffersDataOutput extends DataOutput implements Accountab
     currentBlock = blockAllocate.apply(requiredBlockSize);
     assert currentBlock.capacity() == requiredBlockSize;
     blocks.add(currentBlock);
+    ramBytesUsed += RamUsageEstimator.NUM_BYTES_OBJECT_REF + currentBlock.capacity();
   }
 
   private void rewriteToBlockSize(int targetBlockBits) {
@@ -477,6 +477,7 @@ public final class ByteBuffersDataOutput extends DataOutput implements Accountab
     assert blocks.isEmpty();
     this.blockBits = targetBlockBits;
     blocks.addAll(cloned.blocks);
+    ramBytesUsed = cloned.ramBytesUsed;
   }
 
   private static int computeBlockSizeBitsFor(long bytes) {
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
index 893aa37..deb85f5 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
@@ -20,10 +20,12 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.List;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -154,4 +156,44 @@ public final class TestByteBuffersDataOutput extends BaseDataOutputTestCase<Byte
       assertTrue(bb.hasArray()); // heap-based by default, so array should be there.
     }
   }
+
+  @Test
+  public void testRamBytesUsed() {
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    // Empty output requires no RAM
+    assertEquals(0, out.ramBytesUsed());
+
+    // Non-empty buffer requires RAM
+    out.writeInt(4);
+    assertEquals(out.ramBytesUsed(), computeRamBytesUsed(out));
+
+    // Make sure this keeps working with multiple backing buffers
+    while (out.toBufferList().size() < 2) {
+      out.writeLong(42);
+    }
+    assertEquals(out.ramBytesUsed(), computeRamBytesUsed(out));
+
+    // Make sure this keeps working when increasing the block size
+    int currentBlockCapacity = out.toBufferList().get(0).capacity();
+    do {
+      out.writeLong(42);
+    } while (out.toBufferList().get(0).capacity() == currentBlockCapacity);
+    assertEquals(out.ramBytesUsed(), computeRamBytesUsed(out));
+
+    // Back to zero after a clear
+    out.reset();
+    assertEquals(0, out.ramBytesUsed());
+
+    // And back to non-empty
+    out.writeInt(4);
+    assertEquals(out.ramBytesUsed(), computeRamBytesUsed(out));
+  }
+
+  private static long computeRamBytesUsed(ByteBuffersDataOutput out) {
+    if (out.size() == 0) {
+      return 0;
+    }
+    List<ByteBuffer> buffers = out.toBufferList();
+    return buffers.stream().mapToLong(ByteBuffer::capacity).sum() + buffers.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+  }
 }