You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/11/04 07:07:32 UTC

[GitHub] [lucene] zhaih commented on a diff in pull request #11881: Further optimize DrillSideways scoring

zhaih commented on code in PR #11881:
URL: https://github.com/apache/lucene/pull/11881#discussion_r1013684022


##########
lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java:
##########
@@ -166,89 +160,158 @@ public int score(LeafCollector collector, Bits acceptDocs, int min, int maxDoc)
     return Integer.MAX_VALUE;
   }
 
+  /**
+   * Query-first scoring specialization when there is only one drill-sideways dimension, which is
+   * likely a common scenario.
+   */
+  private void doQueryFirstScoringSingleDim(
+      Bits acceptDocs, LeafCollector collector, DocsAndCost dim) throws IOException {
+    int docID = baseApproximation.docID();
+    while (docID != DocIdSetIterator.NO_MORE_DOCS) {
+      assert docID == baseApproximation.docID();
+
+      if (acceptDocs != null && acceptDocs.get(docID) == false) {
+        docID = baseApproximation.nextDoc();
+        continue;
+      }
+
+      if (baseTwoPhase != null && baseTwoPhase.matches() == false) {
+        docID = baseApproximation.nextDoc();
+        continue;
+      }
+
+      // We have either a near-miss or full match. Check the sideways dim to see which it is:
+      collectDocID = docID;
+      if (advanceIfBehind(docID, dim.approximation) != docID
+          || (dim.twoPhase != null && dim.twoPhase.matches() == false)) {
+        // The sideways dim missed, so we have a "near miss":
+        collectNearMiss(dim.sidewaysLeafCollector);
+      } else {
+        // Hit passed all filters, so it's "real":
+        collectHit(collector, dim);
+      }
+
+      docID = baseApproximation.nextDoc();
+    }
+  }
+
   /**
    * Used when base query is highly constraining vs the drilldowns, or when the docs must be scored
-   * at once (i.e., like BooleanScorer2, not BooleanScorer). In this case we just .next() on base
-   * and .advance() on the dim filters.
+   * at once (i.e., like BooleanScorer2, not BooleanScorer).
    */
   private void doQueryFirstScoring(Bits acceptDocs, LeafCollector collector, DocsAndCost[] dims)
       throws IOException {
     setScorer(collector, ScoreCachingWrappingScorer.wrap(baseScorer));
 
-    List<DocsAndCost> allDims = Arrays.asList(dims);
-    CollectionUtil.timSort(allDims, APPROXIMATION_COMPARATOR);
+    // Specialize the single-dim use-case as we have a more efficient implementation for that:
+    if (dims.length == 1) {
+      doQueryFirstScoringSingleDim(acceptDocs, collector, dims[0]);
+      return;
+    }
+
+    // Sort our sideways dims by approximation cost so we can advance the lower cost ones first:
+    List<DocsAndCost> sidewaysDims = new ArrayList<>(dims.length);
+    sidewaysDims.addAll(List.of(dims));
+    CollectionUtil.timSort(sidewaysDims, APPROXIMATION_COMPARATOR);
 
-    List<DocsAndCost> twoPhaseDims = null;
+    // Maintain (optional) subset of sideways dims that support two-phase iteration, sorted by
+    // matchCost:
+    List<DocsAndCost> sidewaysTwoPhaseDims = null;
     for (DocsAndCost dim : dims) {
       if (dim.twoPhase != null) {
-        if (twoPhaseDims == null) {
-          twoPhaseDims = new ArrayList<>(dims.length);
+        if (sidewaysTwoPhaseDims == null) {
+          sidewaysTwoPhaseDims = new ArrayList<>();
         }
-        twoPhaseDims.add(dim);
+        sidewaysTwoPhaseDims.add(dim);
       }
     }
-    if (twoPhaseDims != null) {
-      CollectionUtil.timSort(twoPhaseDims, TWO_PHASE_COMPARATOR);
+    if (sidewaysTwoPhaseDims != null) {
+      CollectionUtil.timSort(sidewaysTwoPhaseDims, TWO_PHASE_COMPARATOR);
     }
 
+    // We keep track of a "runaway" dimension, which is a previously "near missed" dimension that
+    // has advanced beyond the docID the rest of the dimensions are positioned on. This functions
+    // a bit like the "head" queue in WANDScorer's "min should match" implementation. We use a
+    // single-valued PQ ordered by docID to easily determine the "closest" runaway dim we'll use
+    // for advancing in the case that multiple dim approximations miss.
+    PriorityQueue<DocsAndCost> runawayDim =
+        new PriorityQueue<>(1) {
+          @Override
+          protected boolean lessThan(DocsAndCost a, DocsAndCost b) {
+            return a.approximation.docID() < b.approximation.docID();
+          }
+        };
+
     int docID = baseApproximation.docID();
 
     nextDoc:
-    while (docID != PostingsEnum.NO_MORE_DOCS) {
+    while (docID != DocIdSetIterator.NO_MORE_DOCS) {
       assert docID == baseApproximation.docID();
 
       if (acceptDocs != null && acceptDocs.get(docID) == false) {
         docID = baseApproximation.nextDoc();
         continue;
       }
 
-      DocsAndCost failedDim = null;
-      for (DocsAndCost dim : allDims) {
-        final int dimDocID;
-        if (dim.approximation.docID() < docID) {
-          dimDocID = dim.approximation.advance(docID);
-        } else {
-          dimDocID = dim.approximation.docID();
-        }
-        if (dimDocID != docID) {
-          if (failedDim != null) {
-            int next = Math.min(dimDocID, failedDim.approximation.docID());
+      // If we carried a "runaway" over from the last iteration, see if we've "caught up" yet:
+      DocsAndCost runaway = runawayDim.top();
+      if (runaway != null && runaway.approximation.docID() <= docID) {

Review Comment:
   I'm not sure whether I understand it right, but seems the only advantage of introducing PQ here is we're able to carry over the previous "runaway" or "failed" dimension. But can't we carry that extra dim by using another variable (Or I guess we could even use only one variable, as in the previous version)? Since there might be slightly more overheads by using the PQ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org