You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by kr...@apache.org on 2022/11/22 18:36:48 UTC

[solr] branch main updated: SOLR-16555: SolrIndexSearcher - FilterCache intersections/andNot should not clone bitsets repeatedly (#1184)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 30bf94db62f SOLR-16555: SolrIndexSearcher - FilterCache intersections/andNot should not clone bitsets repeatedly (#1184)
30bf94db62f is described below

commit 30bf94db62fd92354a9c9437eacf884f8fc862d9
Author: Kevin Risden <ri...@users.noreply.github.com>
AuthorDate: Tue Nov 22 13:36:44 2022 -0500

    SOLR-16555: SolrIndexSearcher - FilterCache intersections/andNot should not clone bitsets repeatedly (#1184)
    
    Co-authored-by: David Smiley <ds...@salesforce.com>
---
 solr/CHANGES.txt                                   |   2 +
 .../src/java/org/apache/solr/search/BitDocSet.java |  22 ++-
 .../org/apache/solr/search/MutableBitDocSet.java   | 147 +++++++++++++++++++++
 .../org/apache/solr/search/SolrIndexSearcher.java  |  88 +++++++-----
 4 files changed, 218 insertions(+), 41 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e8e56f7b129..e90b6502ddf 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -76,6 +76,8 @@ Optimizations
 
 * SOLR-16515: Remove synchronized access to cachedOrdMaps in SlowCompositeReaderWrapper (Dennis Berger, Torsten Bøgh Köster, Marco Petris)
 
+* SOLR-16555: SolrIndexSearcher - FilterCache intersections/andNot should not clone bitsets repeatedly (Kevin Risden, David Smiley)
+
 Bug Fixes
 ---------------------
 
diff --git a/solr/core/src/java/org/apache/solr/search/BitDocSet.java b/solr/core/src/java/org/apache/solr/search/BitDocSet.java
index 0e5cdbbb5f2..1ac943bb1e6 100644
--- a/solr/core/src/java/org/apache/solr/search/BitDocSet.java
+++ b/solr/core/src/java/org/apache/solr/search/BitDocSet.java
@@ -194,19 +194,31 @@ public class BitDocSet extends DocSet {
 
   @Override
   public DocSet andNot(DocSet other) {
-    FixedBitSet newbits = bits.clone();
+    FixedBitSet newbits = getFixedBitSetClone();
+    andNot(newbits, other);
+    return new BitDocSet(newbits);
+  }
+
+  /**
+   * Helper method for andNot that takes FixedBitSet and DocSet. This modifies the provided
+   * FixedBitSet to remove all bits contained in the DocSet argument -- equivalent to calling
+   * a.andNot(b), but modifies the state of the FixedBitSet instead of returning a new FixedBitSet.
+   *
+   * @param bits FixedBitSet to operate on
+   * @param other The DocSet to compare to
+   */
+  protected static void andNot(FixedBitSet bits, DocSet other) {
     if (other instanceof BitDocSet) {
-      newbits.andNot(((BitDocSet) other).bits);
+      bits.andNot(((BitDocSet) other).bits);
     } else {
       DocIterator iter = other.iterator();
       while (iter.hasNext()) {
         int doc = iter.nextDoc();
-        if (doc < newbits.length()) {
-          newbits.clear(doc);
+        if (doc < bits.length()) {
+          bits.clear(doc);
         }
       }
     }
-    return new BitDocSet(newbits);
   }
 
   @Override
diff --git a/solr/core/src/java/org/apache/solr/search/MutableBitDocSet.java b/solr/core/src/java/org/apache/solr/search/MutableBitDocSet.java
new file mode 100644
index 00000000000..8b1f7301ed2
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/MutableBitDocSet.java
@@ -0,0 +1,147 @@
+/*
+ * 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.solr.search;
+
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * A {@link BitDocSet} based implementation that mutates the underlying bits for andNot and
+ * intersection. This allows for computing the combinations of sets without duplicating the
+ * underlying array. This MutableBitDocSet should not be cached because it can be modified.
+ *
+ * @since solr 9.2
+ */
+class MutableBitDocSet extends BitDocSet {
+  private MutableBitDocSet(FixedBitSet bits, int size) {
+    super(bits, size);
+  }
+
+  /**
+   * Returns a mutable BitDocSet that is a copy of the provided BitDocSet.
+   *
+   * @param bitDocSet a BitDocSet
+   * @return copy of bitDocSet that is now mutable
+   */
+  public static MutableBitDocSet fromBitDocSet(BitDocSet bitDocSet) {
+    // don't call size() since we just want to pass through the size
+    // instead of computing it if it was already computed
+    return new MutableBitDocSet(bitDocSet.getFixedBitSetClone(), bitDocSet.size);
+  }
+
+  /**
+   * Returns a new BitDocSet with the same bits if the DocSet provided is a MutableBitDocSet.
+   * Otherwise, just returns the provided DocSet.
+   *
+   * @param docSet DocSet to unwrap if it is a MutableBitDocSet
+   * @return Unwrapped DocSet that is not mutable
+   */
+  public static DocSet unwrapIfMutable(DocSet docSet) {
+    if (docSet instanceof MutableBitDocSet) {
+      MutableBitDocSet mutableBitDocSet = (MutableBitDocSet) docSet;
+      // don't call size() since we just want to pass through the size
+      // instead of computing it if it was already computed
+      return new BitDocSet(mutableBitDocSet.getBits(), mutableBitDocSet.size);
+    }
+    return docSet;
+  }
+
+  private void resetSize() {
+    // We need to reset size since we are changing the cardinality of the underlying bits and size
+    // is typically cached. This forces size to be recomputed as needed.
+    this.size = -1;
+  }
+
+  @Override
+  public DocIterator iterator() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Returns the intersection of this set with another set. This mutates the underlying bits so do
+   * not cache the returned bitset.
+   *
+   * @return a DocSet representing the intersection
+   */
+  @Override
+  public DocSet intersection(DocSet other) {
+    // intersection is overloaded in the smaller DocSets to be more
+    // efficient, so dispatch off of it instead.
+    if (!(other instanceof BitDocSet)) {
+      return other.intersection(this);
+    }
+
+    // Operates directly on the underlying bitsets.
+    FixedBitSet newbits = getFixedBitSet();
+    newbits.and(other.getFixedBitSet());
+
+    resetSize();
+    return this;
+  }
+
+  @Override
+  public int intersectionSize(DocSet other) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean intersects(DocSet other) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int unionSize(DocSet other) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int andNotSize(DocSet other) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void addAllTo(FixedBitSet target) {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Returns the documents in this set that are not in the other set. This mutates the underlying
+   * bits so do not cache the returned bitset.
+   *
+   * @return a DocSet representing this AND NOT other
+   */
+  @Override
+  public DocSet andNot(DocSet other) {
+    andNot(getFixedBitSet(), other);
+    resetSize();
+    return this;
+  }
+
+  @Override
+  public DocSet union(DocSet other) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BitDocSet clone() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String toString() {
+    return "MutableBitDocSet instance of " + super.toString();
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 631ebdd0109..7a415c44e35 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -954,21 +954,26 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       return getLiveDocSet();
     }
 
+    DocSet answer;
     if (SolrQueryTimeoutImpl.getInstance().isTimeoutEnabled()) {
       // If there is a possibility of timeout for this query, then don't reserve a computation slot.
       // Further, we can't naively wait for an in progress computation to finish, because if we time
       // out before it does then we won't even have partial results to provide. We could possibly
       // wait for the query to finish in parallel with our own results and if they complete first
       // use that instead, but we'll leave that to implement later.
-      DocSet answer = filterCache.get(query);
-      if (answer != null) {
-        return answer;
+      answer = filterCache.get(query);
+
+      // Not found in the cache so compute and put in the cache
+      if (answer == null) {
+        answer = getDocSetNC(query, null);
+        filterCache.put(query, answer);
       }
-      answer = getDocSetNC(query, null);
-      filterCache.put(query, answer);
-      return answer;
+    } else {
+      answer = filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
     }
-    return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
+
+    assert !(answer instanceof MutableBitDocSet) : "should not be mutable";
+    return answer;
   }
 
   private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
@@ -1151,20 +1156,17 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     // This might become pf.answer but not if there are any non-cached filters
     DocSet answer = null;
 
-    boolean[] neg = new boolean[queries.size() + 1];
-    DocSet[] sets = new DocSet[queries.size() + 1];
+    boolean[] neg = new boolean[queries.size()];
+    DocSet[] sets = new DocSet[queries.size()];
     List<ExtendedQuery> notCached = null;
     List<PostFilter> postFilters = null;
 
-    int end = 0;
-    int smallestIndex = -1;
+    int end = 0; // size of "sets" and "neg"; parallel arrays
 
     if (setFilter != null) {
-      answer = sets[end++] = setFilter;
-      smallestIndex = end;
+      answer = setFilter;
     } // we are done with setFilter at this point
 
-    int smallestCount = Integer.MAX_VALUE;
     for (Query q : queries) {
       if (q instanceof ExtendedQuery) {
         ExtendedQuery eq = (ExtendedQuery) q;
@@ -1190,38 +1192,52 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       }
 
       Query posQuery = QueryUtils.getAbs(q);
-      sets[end] = getPositiveDocSet(posQuery);
+      DocSet docSet = getPositiveDocSet(posQuery);
       // Negative query if absolute value different from original
       if (Objects.equals(q, posQuery)) {
-        neg[end] = false;
-        // keep track of the smallest positive set.
-        // This optimization is only worth it if size() is cached, which it would
-        // be if we don't do any set operations.
-        int sz = sets[end].size();
-        if (sz < smallestCount) {
-          smallestCount = sz;
-          smallestIndex = end;
-          answer = sets[end];
+        // keep track of the smallest positive set; use "answer" for this.
+        if (answer == null) {
+          answer = docSet;
+          continue;
         }
+        // note: assume that size() is cached.  It generally comes from the cache, so should be.
+        if (docSet.size() < answer.size()) {
+          // swap answer & docSet so that answer is smallest
+          DocSet tmp = answer;
+          answer = docSet;
+          docSet = tmp;
+        }
+        neg[end] = false;
       } else {
         neg[end] = true;
       }
+      sets[end++] = docSet;
+    } // end of queries
 
-      end++;
-    }
+    if (end > 0) {
+      // Are all of our normal cached filters negative?
+      if (answer == null) {
+        answer = getLiveDocSet();
+      }
 
-    // Are all of our normal cached filters negative?
-    if (end > 0 && answer == null) {
-      answer = getLiveDocSet();
-    }
+      // This optimizes for the case where we have more than 2 filters and instead
+      // of copying the bitsets we make one mutable bitset. We should only do this
+      // for BitDocSet since it clones the backing bitset for andNot and intersection.
+      if (end > 1 && answer instanceof BitDocSet) {
+        answer = MutableBitDocSet.fromBitDocSet((BitDocSet) answer);
+      }
 
-    // do negative queries first to shrink set size
-    for (int i = 0; i < end; i++) {
-      if (neg[i]) answer = answer.andNot(sets[i]);
-    }
+      // do negative queries first to shrink set size
+      for (int i = 0; i < end; i++) {
+        if (neg[i]) answer = answer.andNot(sets[i]);
+      }
+
+      for (int i = 0; i < end; i++) {
+        if (!neg[i]) answer = answer.intersection(sets[i]);
+      }
 
-    for (int i = 0; i < end; i++) {
-      if (!neg[i] && i != smallestIndex) answer = answer.intersection(sets[i]);
+      // Make sure to keep answer as an immutable DocSet if we made it mutable
+      answer = MutableBitDocSet.unwrapIfMutable(answer);
     }
 
     // ignore "answer" if it simply matches all docs