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/12 15:17:02 UTC

[lucene] branch branch_9x updated: LUCENE-10619: Optimize the writeBytes in TermsHashPerField (#966)

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


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 9f9786122b4 LUCENE-10619: Optimize the writeBytes in TermsHashPerField (#966)
9f9786122b4 is described below

commit 9f9786122b487f992119f45c5d8a51a8d9d4a6f8
Author: tang donghai <ta...@gmail.com>
AuthorDate: Tue Jul 12 23:14:37 2022 +0800

    LUCENE-10619: Optimize the writeBytes in TermsHashPerField (#966)
---
 lucene/CHANGES.txt                                 |  3 ++
 .../org/apache/lucene/index/TermsHashPerField.java | 24 +++++++++++++--
 .../java/org/apache/lucene/util/ByteBlockPool.java | 10 ++++++-
 .../apache/lucene/index/TestTermsHashPerField.java | 25 ++++++++++++++++
 .../org/apache/lucene/util/TestByteBlockPool.java  | 34 ++++++++++++++++++++++
 5 files changed, 93 insertions(+), 3 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 8424b184aca..446294f2c0c 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -33,6 +33,9 @@ Improvements
 * LUCENE-10603: Update SortedSetDocValues iteration to use SortedSetDocValues#docValueCount().
   (Greg Miller, Stefan Vodita)
 
+
+* LUCENE-10619: Optimize the writeBytes in TermsHashPerField. (Tang Donghai)
+
 * GITHUB#983: AbstractSortedSetDocValueFacetCounts internal code cleanup/refactoring. (Greg Miller)
 
 Optimizations
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java b/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
index b8120035c8d..29c135e6cc1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
@@ -230,9 +230,29 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   }
 
   final void writeBytes(int stream, byte[] b, int offset, int len) {
-    // TODO: optimize
     final int end = offset + len;
-    for (int i = offset; i < end; i++) writeByte(stream, b[i]);
+    int streamAddress = streamAddressOffset + stream;
+    int upto = termStreamAddressBuffer[streamAddress];
+    byte[] slice = bytePool.buffers[upto >> ByteBlockPool.BYTE_BLOCK_SHIFT];
+    assert slice != null;
+    int sliceOffset = upto & ByteBlockPool.BYTE_BLOCK_MASK;
+
+    while (slice[sliceOffset] == 0 && offset < end) {
+      slice[sliceOffset++] = b[offset++];
+      (termStreamAddressBuffer[streamAddress])++;
+    }
+
+    while (offset < end) {
+      int offsetAndLength = bytePool.allocKnownSizeSlice(slice, sliceOffset);
+      sliceOffset = offsetAndLength >> 8;
+      int sliceLength = offsetAndLength & 0xff;
+      slice = bytePool.buffer;
+      int writeLength = Math.min(sliceLength - 1, end - offset);
+      System.arraycopy(b, offset, slice, sliceOffset, writeLength);
+      sliceOffset += writeLength;
+      offset += writeLength;
+      termStreamAddressBuffer[streamAddress] = sliceOffset + bytePool.byteOffset;
+    }
   }
 
   final void writeVInt(int stream, int i) {
diff --git a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
index 5bf5ffc9a79..39216b85c31 100644
--- a/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
+++ b/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
@@ -240,7 +240,15 @@ public final class ByteBlockPool implements Accountable {
    * pool.
    */
   public int allocSlice(final byte[] slice, final int upto) {
+    return allocKnownSizeSlice(slice, upto) >> 8;
+  }
 
+  /**
+   * Create a new byte slice with the given starting size return the slice offset in the pool and
+   * length. The lower 8 bits of the returned int represent the length of the slice, and the upper
+   * 24 bits represent the offset.
+   */
+  public int allocKnownSizeSlice(final byte[] slice, final int upto) {
     final int level = slice[upto] & 15;
     final int newLevel = NEXT_LEVEL_ARRAY[level];
     final int newSize = LEVEL_SIZE_ARRAY[newLevel];
@@ -268,7 +276,7 @@ public final class ByteBlockPool implements Accountable {
     // Write new level:
     buffer[byteUpto - 1] = (byte) (16 | newLevel);
 
-    return newUpto + 3;
+    return ((newUpto + 3) << 8) | (newSize - 3);
   }
 
   /**
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermsHashPerField.java b/lucene/core/src/test/org/apache/lucene/index/TestTermsHashPerField.java
index 8711c16fdd0..9c9d39f7092 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTermsHashPerField.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTermsHashPerField.java
@@ -29,6 +29,7 @@ import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
@@ -298,4 +299,28 @@ public class TestTermsHashPerField extends LuceneTestCase {
       assertTrue("the last posting must be EOF on the reader", eof);
     }
   }
+
+  public void testWriteBytes() throws IOException {
+    for (int i = 0; i < 100; i++) {
+      AtomicInteger newCalled = new AtomicInteger(0);
+      AtomicInteger addCalled = new AtomicInteger(0);
+      TermsHashPerField hash = createNewHash(newCalled, addCalled);
+      hash.start(null, true);
+      hash.add(newBytesRef("start"), 0); // tid = 0;
+      int size = TestUtil.nextInt(random(), 50000, 100000);
+      byte[] randomData = new byte[size];
+      random().nextBytes(randomData);
+      int offset = 0;
+      while (offset < randomData.length) {
+        int writeLength = Math.min(randomData.length - offset, TestUtil.nextInt(random(), 1, 200));
+        hash.writeBytes(0, randomData, offset, writeLength);
+        offset += writeLength;
+      }
+      ByteSliceReader reader = new ByteSliceReader();
+      reader.init(hash.bytePool, 0, hash.bytePool.byteOffset + hash.bytePool.byteUpto);
+      for (byte expected : randomData) {
+        assertEquals(expected, reader.readByte());
+      }
+    }
+  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java b/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
index d958204b464..8068d06d72f 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
@@ -101,4 +101,38 @@ public class TestByteBlockPool extends LuceneTestCase {
       position += expected.length;
     }
   }
+
+  public void testAllocKnowSizeSlice() throws IOException {
+    Counter bytesUsed = Counter.newCounter();
+    ByteBlockPool pool = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(bytesUsed));
+    pool.nextBuffer();
+    for (int i = 0; i < 100; i++) {
+      int size;
+      if (random().nextBoolean()) {
+        size = TestUtil.nextInt(random(), 100, 1000);
+      } else {
+        size = TestUtil.nextInt(random(), 50000, 100000);
+      }
+      byte[] randomData = new byte[size];
+      random().nextBytes(randomData);
+
+      int upto = pool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
+
+      for (int offset = 0; offset < size; ) {
+        if ((pool.buffer[upto] & 16) == 0) {
+          pool.buffer[upto++] = randomData[offset++];
+        } else {
+          int offsetAndLength = pool.allocKnownSizeSlice(pool.buffer, upto);
+          int sliceLength = offsetAndLength & 0xff;
+          upto = offsetAndLength >> 8;
+          assertNotEquals(0, pool.buffer[upto + sliceLength - 1]);
+          assertEquals(0, pool.buffer[upto]);
+          int writeLength = Math.min(sliceLength - 1, size - offset);
+          System.arraycopy(randomData, offset, pool.buffer, upto, writeLength);
+          offset += writeLength;
+          upto += writeLength;
+        }
+      }
+    }
+  }
 }