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/10/18 09:07:59 UTC

[lucene] branch main updated: Binary search the entries when all suffixes have the same length in a leaf block. (#11722)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 3adec5b1ce9 Binary search the entries when all suffixes have the same length in a leaf block. (#11722)
3adec5b1ce9 is described below

commit 3adec5b1ce94cc6e910375e01d270373a8cebba3
Author: zhouhui <vs...@163.com>
AuthorDate: Tue Oct 18 17:07:52 2022 +0800

    Binary search the entries when all suffixes have the same length in a leaf block. (#11722)
---
 lucene/CHANGES.txt                                 |  3 +
 .../lucene90/blocktree/SegmentTermsEnumFrame.java  | 99 +++++++++++++++++++++-
 .../tests/index/BasePostingsFormatTestCase.java    | 51 +++++++++++
 3 files changed, 149 insertions(+), 4 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 00e140ee713..4c1fbb84960 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -112,6 +112,9 @@ Improvements
 
 * GITHUB#687: speed up IndexSortSortedNumericDocValuesRangeQuery#BoundedDocIdSetIterator
   construction using bkd binary search. (Jianping Weng)
+  
+* GITHUB#11722: Binary search the entries when all suffixes have the same length
+  in a leaf block of the terms dictionary. (zhouhui)
 
 Bug Fixes
 ---------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java
index 48c4fd0a6d4..d9a28c51895 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java
@@ -75,6 +75,9 @@ final class SegmentTermsEnumFrame {
   // True if all entries are terms
   boolean isLeafBlock;
 
+  // True if all entries have the same length.
+  boolean allEqual;
+
   long lastSubFP;
 
   int nextFloorLabel;
@@ -187,7 +190,7 @@ final class SegmentTermsEnumFrame {
     suffixesReader.reset(suffixBytes, 0, numSuffixBytes);
 
     int numSuffixLengthBytes = ste.in.readVInt();
-    final boolean allEqual = (numSuffixLengthBytes & 0x01) != 0;
+    allEqual = (numSuffixLengthBytes & 0x01) != 0;
     numSuffixLengthBytes >>>= 1;
     if (suffixLengthBytes.length < numSuffixLengthBytes) {
       suffixLengthBytes = new byte[ArrayUtil.oversize(numSuffixLengthBytes, 1)];
@@ -527,7 +530,9 @@ final class SegmentTermsEnumFrame {
 
   // NOTE: sets startBytePos/suffix as a side effect
   public SeekStatus scanToTerm(BytesRef target, boolean exactOnly) throws IOException {
-    return isLeafBlock ? scanToTermLeaf(target, exactOnly) : scanToTermNonLeaf(target, exactOnly);
+    return isLeafBlock
+        ? allEqual ? binarySearchTermLeaf(target, exactOnly) : scanToTermLeaf(target, exactOnly)
+        : scanToTermNonLeaf(target, exactOnly);
   }
 
   private int startBytePos;
@@ -572,8 +577,6 @@ final class SegmentTermsEnumFrame {
 
     assert prefixMatches(target);
 
-    // TODO: binary search when all terms have the same length, which is common for ID fields,
-    // which are also the most sensitive to lookup performance?
     // Loop over each entry (term or sub-block) in this block:
     do {
       nextEnt++;
@@ -646,6 +649,94 @@ final class SegmentTermsEnumFrame {
     return SeekStatus.END;
   }
 
+  // Target's prefix matches this block's prefix;
+  // And all suffixes have the same length in this block,
+  // we binary search the entries check if the suffix matches.
+  public SeekStatus binarySearchTermLeaf(BytesRef target, boolean exactOnly) throws IOException {
+    // if (DEBUG) System.out.println("    binarySearchTermLeaf: block fp=" + fp + " prefix=" +
+    // prefix + "
+    // nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" +
+    // brToString(term));
+
+    assert nextEnt != -1;
+
+    ste.termExists = true;
+    subCode = 0;
+
+    if (nextEnt == entCount) {
+      if (exactOnly) {
+        fillTerm();
+      }
+      return SeekStatus.END;
+    }
+
+    assert prefixMatches(target);
+
+    suffix = suffixLengthsReader.readVInt();
+    int start = nextEnt;
+    int end = entCount - 1;
+    // Binary search the entries (terms) in this leaf block:
+    int cmp = 0;
+    while (start <= end) {
+      int mid = (start + end) / 2;
+      nextEnt = mid + 1;
+      startBytePos = mid * suffix;
+      suffixesReader.setPosition(startBytePos + suffix);
+
+      // Binary search bytes in the suffix, comparing to the target
+      cmp =
+          Arrays.compareUnsigned(
+              suffixBytes,
+              startBytePos,
+              startBytePos + suffix,
+              target.bytes,
+              target.offset + prefix,
+              target.offset + target.length);
+      if (cmp < 0) {
+        start = mid + 1;
+      } else if (cmp > 0) {
+        end = mid - 1;
+      } else {
+        // Exact match!
+
+        // This cannot be a sub-block because we
+        // would have followed the index to this
+        // sub-block from the start:
+
+        assert ste.termExists;
+        fillTerm();
+        // if (DEBUG) System.out.println("        found!");
+        return SeekStatus.FOUND;
+      }
+    }
+
+    // It is possible (and OK) that terms index pointed us
+    // at this block, but, we searched the entire block and
+    // did not find the term to position to.  This happens
+    // when the target is after the last term in the block
+    // (but, before the next term in the index).  EG
+    // target could be foozzz, and terms index pointed us
+    // to the foo* block, but the last term in this block
+    // was fooz (and, eg, first term in the next block will
+    // bee fop).
+    // if (DEBUG) System.out.println("      block end");
+    SeekStatus seekStatus = end < entCount - 1 ? SeekStatus.NOT_FOUND : SeekStatus.END;
+    if (exactOnly || seekStatus == SeekStatus.NOT_FOUND) {
+      // If binary search ended at the less term, we need to advance to the greater term.
+      if (cmp < 0) {
+        startBytePos += suffix;
+        suffixesReader.skipBytes(suffix);
+        nextEnt++;
+      }
+      fillTerm();
+    }
+
+    // TODO: not consistent that in the
+    // not-exact case we don't next() into the next
+    // frame here
+    return seekStatus;
+  }
+
   // Target's prefix matches this block's prefix; we
   // scan the entries check if the suffix matches.
   public SeekStatus scanToTermNonLeaf(BytesRef target, boolean exactOnly) throws IOException {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java
index d98fa17b974..16480692ff8 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java
@@ -367,6 +367,57 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
     dir.close();
   }
 
+  public void testBinarySearchTermLeaf() throws Exception {
+    Directory dir = newDirectory();
+
+    IndexWriterConfig iwc = newIndexWriterConfig(null);
+    iwc.setCodec(getCodec());
+    iwc.setMergePolicy(newTieredMergePolicy());
+    IndexWriter iw = new IndexWriter(dir, iwc);
+
+    for (int i = 100000; i <= 100400; i++) {
+      // only add odd number
+      if (i % 2 == 1) {
+        Document document = new Document();
+        document.add(new StringField("id", i + "", Field.Store.NO));
+        iw.addDocument(document);
+      }
+    }
+    iw.commit();
+    iw.forceMerge(1);
+
+    DirectoryReader reader = DirectoryReader.open(iw);
+    TermsEnum termsEnum = getOnlyLeafReader(reader).terms("id").iterator();
+    // test seekExact
+    for (int i = 100000; i <= 100400; i++) {
+      BytesRef target = new BytesRef(i + "");
+      if (i % 2 == 1) {
+        assertTrue(termsEnum.seekExact(target));
+        assertEquals(termsEnum.term(), target);
+      } else {
+        assertFalse(termsEnum.seekExact(target));
+      }
+    }
+    // test seekCeil
+    for (int i = 100000; i < 100400; i++) {
+      BytesRef target = new BytesRef(i + "");
+      if (i % 2 == 1) {
+        assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(target));
+        assertEquals(termsEnum.term(), target);
+        if (i <= 100397) {
+          assertEquals(new BytesRef(i + 2 + ""), termsEnum.next());
+        }
+      } else {
+        assertEquals(SeekStatus.NOT_FOUND, termsEnum.seekCeil(target));
+        assertEquals(new BytesRef(i + 1 + ""), termsEnum.term());
+      }
+    }
+    assertEquals(SeekStatus.END, termsEnum.seekCeil(new BytesRef(100400 + "")));
+    reader.close();
+    iw.close();
+    dir.close();
+  }
+
   // tests that level 2 ghost fields still work
   public void testLevel2Ghosts() throws Exception {
     Directory dir = newDirectory();