You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gs...@apache.org on 2022/07/19 21:27:28 UTC

[lucene] branch branch_9x updated: LUCENE-10653: Heapify in BMMScorer (#1022)

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

gsmiller 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 42729b46c48 LUCENE-10653: Heapify in BMMScorer (#1022)
42729b46c48 is described below

commit 42729b46c48308405f0575aa41cc655f94b549f0
Author: Greg Miller <gs...@gmail.com>
AuthorDate: Tue Jul 19 13:49:31 2022 -0700

    LUCENE-10653: Heapify in BMMScorer (#1022)
---
 lucene/CHANGES.txt                                 |   2 +
 .../lucene/search/BlockMaxMaxscoreScorer.java      |  16 ++-
 .../apache/lucene/search/DisiPriorityQueue.java    |  42 ++++++
 .../lucene/search/TestDisiPriorityQueue.java       | 157 +++++++++++++++++++++
 4 files changed, 210 insertions(+), 7 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 55cbb9bfdc8..b9e6eb4964b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -65,6 +65,8 @@ Optimizations
 * GITHUB#1007: Optimize IntersectVisitor#visit implementations for certain bulk-add cases.
   (Greg Miller)
 
+* LUCENE-10653: BlockMaxMaxscoreScorer uses heapify instead of individual adds. (Greg Miller)
+
 Changes in runtime behavior
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/search/BlockMaxMaxscoreScorer.java b/lucene/core/src/java/org/apache/lucene/search/BlockMaxMaxscoreScorer.java
index f6305e5e2b3..c6fa22a66e0 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BlockMaxMaxscoreScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BlockMaxMaxscoreScorer.java
@@ -158,9 +158,10 @@ class BlockMaxMaxscoreScorer extends Scorer {
             // list adjusted
             if (removedEssentialScorer) {
               essentialsScorers.clear();
-              for (int i = firstEssentialScorerIndex; i < allScorers.length; ++i) {
-                essentialsScorers.add(allScorers[i]);
-              }
+              essentialsScorers.addAll(
+                  allScorers,
+                  firstEssentialScorerIndex,
+                  allScorers.length - firstEssentialScorerIndex);
             }
           }
 
@@ -199,7 +200,6 @@ class BlockMaxMaxscoreScorer extends Scorer {
           }
 
           private void repartitionLists() {
-            essentialsScorers.clear();
             firstEssentialScorerIndex = 0;
             Arrays.sort(allScorers, Comparator.comparingDouble(scorer -> scorer.maxScore));
 
@@ -214,9 +214,11 @@ class BlockMaxMaxscoreScorer extends Scorer {
               firstEssentialScorerIndex++;
               nonEssentialMaxScoreSum += w.maxScore;
             }
-            for (int i = firstEssentialScorerIndex; i < allScorers.length; ++i) {
-              essentialsScorers.add(allScorers[i]);
-            }
+            essentialsScorers.clear();
+            essentialsScorers.addAll(
+                allScorers,
+                firstEssentialScorerIndex,
+                allScorers.length - firstEssentialScorerIndex);
           }
 
           @Override
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisiPriorityQueue.java b/lucene/core/src/java/org/apache/lucene/search/DisiPriorityQueue.java
index 2d9a3e0d8b2..2d5ea830d33 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisiPriorityQueue.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisiPriorityQueue.java
@@ -103,6 +103,48 @@ public final class DisiPriorityQueue implements Iterable<DisiWrapper> {
     return heap[0];
   }
 
+  public void addAll(DisiWrapper[] entries, int offset, int len) {
+    // Nothing to do if empty:
+    if (len == 0) {
+      return;
+    }
+
+    // Fail early if we're going to over-fill:
+    if (size + len > heap.length) {
+      throw new IndexOutOfBoundsException(
+          "Cannot add "
+              + len
+              + " elements to a queue with remaining capacity "
+              + (heap.length - size));
+    }
+
+    // Copy the entries over to our heap array:
+    System.arraycopy(entries, offset, heap, size, len);
+    size += len;
+
+    // Heapify in bulk:
+    final int firstLeafIndex = size >>> 1;
+    for (int rootIndex = firstLeafIndex - 1; rootIndex >= 0; rootIndex--) {
+      int parentIndex = rootIndex;
+      DisiWrapper parent = heap[parentIndex];
+      while (parentIndex < firstLeafIndex) {
+        int childIndex = leftNode(parentIndex);
+        int rightChildIndex = rightNode(childIndex);
+        DisiWrapper child = heap[childIndex];
+        if (rightChildIndex < size && heap[rightChildIndex].doc < child.doc) {
+          child = heap[rightChildIndex];
+          childIndex = rightChildIndex;
+        }
+        if (child.doc >= parent.doc) {
+          break;
+        }
+        heap[parentIndex] = child;
+        parentIndex = childIndex;
+      }
+      heap[parentIndex] = parent;
+    }
+  }
+
   public DisiWrapper pop() {
     final DisiWrapper[] heap = this.heap;
     final DisiWrapper result = heap[0];
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDisiPriorityQueue.java b/lucene/core/src/test/org/apache/lucene/search/TestDisiPriorityQueue.java
new file mode 100644
index 00000000000..94daa2fad71
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDisiPriorityQueue.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.PrimitiveIterator.OfInt;
+import java.util.Random;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.tests.util.LuceneTestCase;
+
+public class TestDisiPriorityQueue extends LuceneTestCase {
+
+  public void testRandom() throws Exception {
+    Random r = random();
+
+    int size = r.nextInt(TEST_NIGHTLY ? 1000 : 10);
+    DisiWrapper[] all = new DisiWrapper[size];
+    for (int i = 0; i < size; i++) {
+      DocIdSetIterator it = randomDisi(r);
+      DisiWrapper w = wrapper(it);
+      all[i] = w;
+    }
+
+    DisiPriorityQueue pq = new DisiPriorityQueue(size);
+    if (r.nextBoolean()) {
+      for (DisiWrapper w : all) {
+        pq.add(w);
+      }
+    } else {
+      if (r.nextInt(10) < 2) {
+        int len = RandomNumbers.randomIntBetween(random(), 1, size);
+        for (int i = 0; i < len; i++) {
+          pq.add(all[i]);
+        }
+        pq.addAll(all, len, size - len);
+      } else {
+        pq.addAll(all, 0, size);
+      }
+    }
+
+    while (pq.size() > 0) {
+      Arrays.sort(all, Comparator.comparingInt(w -> w.doc));
+      DisiWrapper top = pq.top();
+      assertEquals(all[0].doc, top.doc);
+      top.doc = top.iterator.nextDoc();
+      if (top.doc == DocIdSetIterator.NO_MORE_DOCS) {
+        pq.pop();
+      } else {
+        pq.updateTop();
+      }
+    }
+  }
+
+  private static DisiWrapper wrapper(DocIdSetIterator iterator) throws IOException {
+    Query q = new DummyQuery(iterator);
+    Scorer s = q.createWeight(null, ScoreMode.COMPLETE_NO_SCORES, 1.0f).scorer(null);
+    return new DisiWrapper(s);
+  }
+
+  private static DocIdSetIterator randomDisi(Random r) {
+    int maxSize = r.nextInt(50);
+    OfInt randomInts =
+        r.ints(maxSize, 0, DocIdSetIterator.NO_MORE_DOCS - 1).sorted().distinct().iterator();
+    return new DocIdSetIterator() {
+      private int doc = -1;
+
+      @Override
+      public int docID() {
+        return doc;
+      }
+
+      @Override
+      public int nextDoc() {
+        if (randomInts.hasNext()) {
+          return doc = randomInts.nextInt();
+        } else {
+          return doc = DocIdSetIterator.NO_MORE_DOCS;
+        }
+      }
+
+      @Override
+      public int advance(int target) {
+        while (doc < target) {
+          nextDoc();
+        }
+        return doc;
+      }
+
+      @Override
+      public long cost() {
+        return maxSize;
+      }
+    };
+  }
+
+  private static class DummyQuery extends Query {
+    private static int COUNTER = 0;
+    private final int id;
+    private final DocIdSetIterator disi;
+
+    DummyQuery(DocIdSetIterator disi) {
+      id = COUNTER++;
+      this.disi = disi;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost)
+        throws IOException {
+      return new ConstantScoreWeight(this, boost) {
+        @Override
+        public Scorer scorer(LeafReaderContext context) {
+          return new ConstantScoreScorer(this, score(), scoreMode, disi);
+        }
+
+        @Override
+        public boolean isCacheable(LeafReaderContext ctx) {
+          return true;
+        }
+      };
+    }
+
+    @Override
+    public String toString(String field) {
+      return "DummyQuery (" + id + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {}
+
+    @Override
+    public boolean equals(Object other) {
+      return sameClassAs(other) && id == ((DummyQuery) other).id;
+    }
+
+    @Override
+    public int hashCode() {
+      return id;
+    }
+  }
+}