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 2023/01/12 17:39:41 UTC

[lucene] branch branch_9x updated (eaa2d3180c2 -> 5f0d2207344)

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

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


    from eaa2d3180c2 Fix exponential runtime for Boolean#rewrite (#12072)
     new 40350f71c14 Speed up 1D BKD merging. (#12079)
     new 5f0d2207344 Speed up DocIdMerger on sorted indexes. (#12081)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 lucene/CHANGES.txt                                  |  4 ++++
 .../java/org/apache/lucene/index/DocIDMerger.java   | 21 ++++++++++++++++++++-
 .../java/org/apache/lucene/util/bkd/BKDWriter.java  |  8 ++++----
 3 files changed, 28 insertions(+), 5 deletions(-)


[lucene] 01/02: Speed up 1D BKD merging. (#12079)

Posted by jp...@apache.org.
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 40350f71c14d5a8c99b356ab0b827c9552adf381
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu Jan 12 18:14:15 2023 +0100

    Speed up 1D BKD merging. (#12079)
    
    On the NYC taxis dataset on my local machine, switching from
    `Arrays#compareUnsigned` to `ArrayUtil#getUnsignedComparator` yielded a 15%
    speedup of BKD merging.
---
 lucene/CHANGES.txt                                             | 2 ++
 lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java | 8 ++++----
 2 files changed, 6 insertions(+), 4 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 3ffd6594bd2..97f1e833019 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -182,6 +182,8 @@ Optimizations
 
 * GITHUB#12017: Aggressive count in BooleanWeight. (Lu Xugang)
 
+* GITHUB#12079: Faster merging of 1D points. (Adrien Grand)
+
 
 Other
 ---------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 025342400f3..51c10564ba9 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -367,19 +367,19 @@ public class BKDWriter implements Closeable {
   }
 
   private static class BKDMergeQueue extends PriorityQueue<MergeReader> {
-    private final int bytesPerDim;
+    private final ArrayUtil.ByteArrayComparator comparator;
 
     public BKDMergeQueue(int bytesPerDim, int maxSize) {
       super(maxSize);
-      this.bytesPerDim = bytesPerDim;
+      this.comparator = ArrayUtil.getUnsignedComparator(bytesPerDim);
     }
 
     @Override
     public boolean lessThan(MergeReader a, MergeReader b) {
       assert a != b;
 
-      int cmp =
-          Arrays.compareUnsigned(a.packedValue, 0, bytesPerDim, b.packedValue, 0, bytesPerDim);
+      int cmp = comparator.compare(a.packedValue, 0, b.packedValue, 0);
+
       if (cmp < 0) {
         return true;
       } else if (cmp > 0) {


[lucene] 02/02: Speed up DocIdMerger on sorted indexes. (#12081)

Posted by jp...@apache.org.
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 5f0d22073446d174a37f80a6d2265b36f4bf247c
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu Jan 12 18:27:45 2023 +0100

    Speed up DocIdMerger on sorted indexes. (#12081)
    
    In the case when an index is sorted on a low-cardinality field, or the index
    sort order correlates with the order in which documents get ingested, we can
    optimize `SortedDocIDMerger` by doing a single comparison with the doc ID on
    the next sub. This checks covers at the same time whether the priority queue
    needs reordering and whether the current sub reached `NO_MORE_DOCS`.
---
 lucene/CHANGES.txt                                  |  2 ++
 .../java/org/apache/lucene/index/DocIDMerger.java   | 21 ++++++++++++++++++++-
 2 files changed, 22 insertions(+), 1 deletion(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 97f1e833019..5dd8547bf58 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -184,6 +184,8 @@ Optimizations
 
 * GITHUB#12079: Faster merging of 1D points. (Adrien Grand)
 
+* GITHUB#12081: Small merging speedup on sorted indexes. (Adrien Grand)
+
 
 Other
 ---------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
index f9accb710f1..dd1840ca850 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
@@ -137,6 +137,7 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
     private final List<T> subs;
     private T current;
     private final PriorityQueue<T> queue;
+    private int queueMinDocID;
 
     private SortedDocIDMerger(List<T> subs, int maxCount) throws IOException {
       if (maxCount <= 1) {
@@ -154,6 +155,14 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
       reset();
     }
 
+    private void setQueueMinDocID() {
+      if (queue.size() > 0) {
+        queueMinDocID = queue.top().mappedDocID;
+      } else {
+        queueMinDocID = DocIdSetIterator.NO_MORE_DOCS;
+      }
+    }
+
     @Override
     public void reset() throws IOException {
       // caller may not have fully consumed the queue:
@@ -171,23 +180,33 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
           queue.add(sub);
         } // else all docs in this sub were deleted; do not add it to the queue!
       }
+      setQueueMinDocID();
     }
 
     @Override
     public T next() throws IOException {
       int nextDoc = current.nextMappedDoc();
+      if (nextDoc < queueMinDocID) {
+        // This should be the common case when index sorting is either disabled, or enabled on a
+        // low-cardinality field, or enabled on a field that correlates with index order.
+        return current;
+      }
+
       if (nextDoc == NO_MORE_DOCS) {
         if (queue.size() == 0) {
           current = null;
         } else {
           current = queue.pop();
         }
-      } else if (queue.size() > 0 && nextDoc > queue.top().mappedDocID) {
+      } else if (queue.size() > 0) {
+        assert queueMinDocID == queue.top().mappedDocID;
+        assert nextDoc > queueMinDocID;
         T newCurrent = queue.top();
         queue.updateTop(current);
         current = newCurrent;
       }
 
+      setQueueMinDocID();
       return current;
     }
   }