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 2021/07/29 07:55:41 UTC

[lucene-solr] branch branch_8x updated: LUCENE-10031: Speed up SortedDocIdMerger on low-cardinality sort fields. (#221)

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 4246488  LUCENE-10031: Speed up SortedDocIdMerger on low-cardinality sort fields. (#221)
4246488 is described below

commit 42464888c6b42b56b2d20ceb8e65fb08e600f5aa
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu Jul 29 08:52:44 2021 +0200

    LUCENE-10031: Speed up SortedDocIdMerger on low-cardinality sort fields. (#221)
    
    When sorting by low-cardinality fields, the same sub remains current for long
    sequences of doc IDs. This speeds up SortedDocIdMerger a bit by extracting
    the sub that leads iteration.
---
 lucene/CHANGES.txt                                 |   3 +
 .../java/org/apache/lucene/index/DocIDMerger.java  | 101 ++++++++++-----------
 2 files changed, 49 insertions(+), 55 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index db68902..0bb3cd4 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -62,6 +62,9 @@ Optimizations
 * LUCENE-10022: Rewrite empty DisjunctionMaxQuery to MatchNoDocsQuery.
   (David Harsha via Julie Tibshirani)
 
+* LUCENE-10031: Slightly faster segment merging for sorted indices.
+  (Adrien Grand)
+
 Bug Fixes
 ---------------------
 
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 1ef04c2..f334626 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
@@ -45,6 +45,24 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
 
     /** Returns the next document ID from this sub reader, and {@link DocIdSetIterator#NO_MORE_DOCS} when done */
     public abstract int nextDoc() throws IOException;
+
+    /**
+     * Like {@link #nextDoc()} but skips over unmapped docs and returns the next mapped doc ID, or
+     * {@link DocIdSetIterator#NO_MORE_DOCS} when exhausted. This method sets {@link #mappedDocID}
+     * as a side effect.
+     */
+    public final int nextMappedDoc() throws IOException {
+      while (true) {
+        int doc = nextDoc();
+        if (doc == NO_MORE_DOCS) {
+          return this.mappedDocID = NO_MORE_DOCS;
+        }
+        int mappedDoc = docMap.get(doc);
+        if (mappedDoc != -1) {
+          return this.mappedDocID = mappedDoc;
+        }
+      }
+    }
   }
 
   /** Construct this from the provided subs, specifying the maximum sub count */
@@ -95,24 +113,15 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
 
     @Override
     public T next() throws IOException {
-      while (true) {
-        int docID = current.nextDoc();
-        if (docID == NO_MORE_DOCS) {
-          if (nextIndex == subs.size()) {
-            current = null;
-            return null;
-          }
-          current = subs.get(nextIndex);
-          nextIndex++;
-          continue;
-        }
-
-        int mappedDocID = current.docMap.get(docID);
-        if (mappedDocID != -1) {
-          current.mappedDocID = mappedDocID;
-          return current;
+      while (current.nextMappedDoc() == NO_MORE_DOCS) {
+        if (nextIndex == subs.size()) {
+          current = null;
+          return null;
         }
+        current = subs.get(nextIndex);
+        nextIndex++;
       }
+      return current;
     }
 
   }
@@ -120,11 +129,15 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
   private static class SortedDocIDMerger<T extends DocIDMerger.Sub> extends DocIDMerger<T> {
 
     private final List<T> subs;
+    private T current;
     private final PriorityQueue<T> queue;
 
     private SortedDocIDMerger(List<T> subs, int maxCount) throws IOException {
+      if (maxCount <= 1) {
+        throw new IllegalArgumentException();
+      }
       this.subs = subs;
-      queue = new PriorityQueue<T>(maxCount) {
+      queue = new PriorityQueue<T>(maxCount - 1) {
         @Override
         protected boolean lessThan(Sub a, Sub b) {
           assert a.mappedDocID != b.mappedDocID;
@@ -138,59 +151,37 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
     public void reset() throws IOException {
       // caller may not have fully consumed the queue:
       queue.clear();
+      current = null;
       boolean first = true;
-      for(T sub : subs) {
+      for (T sub : subs) {
         if (first) {
           // by setting mappedDocID = -1, this entry is guaranteed to be the top of the queue
           // so the first call to next() will advance it
           sub.mappedDocID = -1;
+          current = sub;
           first = false;
-        } else {
-          int mappedDocID;
-          while (true) {
-            int docID = sub.nextDoc();
-            if (docID == NO_MORE_DOCS) {
-              mappedDocID = NO_MORE_DOCS;
-              break;
-            }
-            mappedDocID = sub.docMap.get(docID);
-            if (mappedDocID != -1) {
-              break;
-            }
-          }
-          if (mappedDocID == NO_MORE_DOCS) {
-            // all docs in this sub were deleted; do not add it to the queue!
-            continue;
-          }
-          sub.mappedDocID = mappedDocID;
-        }
-        queue.add(sub);
+        } else if (sub.nextMappedDoc() != NO_MORE_DOCS) {
+          queue.add(sub);
+        } // else all docs in this sub were deleted; do not add it to the queue!
       }
     }
 
     @Override
     public T next() throws IOException {
-      T top = queue.top();
-
-      while (true) {
-        int docID = top.nextDoc();
-        if (docID == NO_MORE_DOCS) {
-          queue.pop();
-          top = queue.top();
-          break;
-        }
-        int mappedDocID = top.docMap.get(docID);
-        if (mappedDocID == -1) {
-          // doc was deleted
-          continue;
+      int nextDoc = current.nextMappedDoc();
+      if (nextDoc == NO_MORE_DOCS) {
+        if (queue.size() == 0) {
+          current = null;
         } else {
-          top.mappedDocID = mappedDocID;
-          top = queue.updateTop();
-          break;
+          current = queue.pop();
         }
+      } else if (queue.size() > 0 && nextDoc > queue.top().mappedDocID) {
+        T newCurrent = queue.top();
+        queue.updateTop(current);
+        current = newCurrent;
       }
 
-      return top;
+      return current;
     }
   }