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 2022/07/27 14:20:58 UTC

[lucene] 01/02: LUCENE-10661: Reduce memory copy in BytesStore (#1047)

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

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

commit 169af9c6511576486bad249f702ce63b2d1ed3ab
Author: luyuncheng <lu...@bytedance.com>
AuthorDate: Wed Jul 27 22:17:08 2022 +0800

    LUCENE-10661: Reduce memory copy in BytesStore (#1047)
---
 lucene/CHANGES.txt                                 |  2 +
 .../org/apache/lucene/util/fst/BytesStore.java     | 45 ++++++++++------------
 .../org/apache/lucene/util/fst/OnHeapFSTStore.java |  3 +-
 .../org/apache/lucene/util/fst/TestBytesStore.java | 23 ++++++++++-
 4 files changed, 46 insertions(+), 27 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 8a958921d05..b6d41c3448b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -93,6 +93,8 @@ Optimizations
 
 * LUCENE-10653: BlockMaxMaxscoreScorer uses heapify instead of individual adds. (Greg Miller)
 
+* LUCENE-10661: Reduce memory copy in BytesStore. (luyuncheng)
+
 Changes in runtime behavior
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
index 2278c27d10b..db9e5cfce10 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
@@ -49,30 +49,6 @@ class BytesStore extends DataOutput implements Accountable {
     nextWrite = blockSize;
   }
 
-  /** Pulls bytes from the provided IndexInput. */
-  public BytesStore(DataInput in, long numBytes, int maxBlockSize) throws IOException {
-    int blockSize = 2;
-    int blockBits = 1;
-    while (blockSize < numBytes && blockSize < maxBlockSize) {
-      blockSize *= 2;
-      blockBits++;
-    }
-    this.blockBits = blockBits;
-    this.blockSize = blockSize;
-    this.blockMask = blockSize - 1;
-    long left = numBytes;
-    while (left > 0) {
-      final int chunk = (int) Math.min(blockSize, left);
-      byte[] block = new byte[chunk];
-      in.readBytes(block, 0, block.length);
-      blocks.add(block);
-      left -= chunk;
-    }
-
-    // So .getPosition still works
-    nextWrite = blocks.get(blocks.size() - 1).length;
-  }
-
   /** Absolute write byte; you must ensure dest is &lt; max position written so far. */
   public void writeByte(long dest, byte b) {
     int blockIndex = (int) (dest >> blockBits);
@@ -179,6 +155,27 @@ class BytesStore extends DataOutput implements Accountable {
     }
   }
 
+  @Override
+  public void copyBytes(DataInput input, long numBytes) throws IOException {
+    assert numBytes >= 0 : "numBytes=" + numBytes;
+    assert input != null;
+    long len = numBytes;
+    while (len > 0) {
+      int chunk = blockSize - nextWrite;
+      int l = (int) Math.min(chunk, len);
+      if (l > 0) {
+        assert current != null;
+        input.readBytes(current, nextWrite, l);
+        nextWrite += l;
+        len -= l;
+      } else {
+        current = new byte[blockSize];
+        blocks.add(current);
+        nextWrite = 0;
+      }
+    }
+  }
+
   /**
    * Absolute copy bytes self to self, without changing the position. Note: this cannot "grow" the
    * bytes, so must only call it on already written parts.
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
index 522fa31f6b6..be33c73fc41 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
@@ -54,7 +54,8 @@ public final class OnHeapFSTStore implements FSTStore {
   public void init(DataInput in, long numBytes) throws IOException {
     if (numBytes > 1 << this.maxBlockBits) {
       // FST is big: we need multiple pages
-      bytes = new BytesStore(in, numBytes, 1 << this.maxBlockBits);
+      bytes = new BytesStore(this.maxBlockBits);
+      bytes.copyBytes(in, numBytes);
     } else {
       // FST fits into a single block: use ByteArrayBytesStoreReader for less overhead
       bytesArray = new byte[(int) numBytes];
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java
index 20a4e918e6b..3b644309c0a 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java
@@ -16,13 +16,16 @@
  */
 package org.apache.lucene.util.fst;
 
+import java.io.IOException;
 import java.util.Arrays;
+import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.ArrayUtil;
 
 public class TestBytesStore extends LuceneTestCase {
 
@@ -224,8 +227,8 @@ public class TestBytesStore extends LuceneTestCase {
         bytes.writeTo(out);
         out.close();
         IndexInput in = dir.openInput("bytes", IOContext.DEFAULT);
-        bytesToVerify =
-            new BytesStore(in, numBytes, TestUtil.nextInt(random(), 256, Integer.MAX_VALUE));
+        bytesToVerify = new BytesStore(TestUtil.nextInt(random(), 8, 20));
+        bytesToVerify.copyBytes(in, numBytes);
         in.close();
         dir.close();
       } else {
@@ -236,6 +239,22 @@ public class TestBytesStore extends LuceneTestCase {
     }
   }
 
+  public void testCopyBytesOnByteStore() throws IOException {
+    byte[] bytes = new byte[1024 * 8 + 10];
+    byte[] bytesout = new byte[bytes.length];
+    random().nextBytes(bytes);
+    int offset = TestUtil.nextInt(random(), 0, 100);
+    int len = bytes.length - offset;
+    ByteArrayDataInput in = new ByteArrayDataInput(bytes, offset, len);
+    final int blockBits = TestUtil.nextInt(random(), 8, 15);
+    final BytesStore o = new BytesStore(blockBits);
+    o.copyBytes(in, len);
+    o.copyBytes(0, bytesout, 0, len);
+    assertArrayEquals(
+        ArrayUtil.copyOfSubArray(bytesout, 0, len),
+        ArrayUtil.copyOfSubArray(bytes, offset, offset + len));
+  }
+
   private void verify(BytesStore bytes, byte[] expected, int totalLength) throws Exception {
     assertEquals(totalLength, bytes.getPosition());
     if (totalLength == 0) {