You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2022/02/02 20:16:53 UTC

[GitHub] [solr] magibney opened a new pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

magibney opened a new pull request #592:
URL: https://github.com/apache/solr/pull/592


   See: [SOLR-14765](https://issues.apache.org/jira/browse/SOLR-14765)


-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799881581



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+      switch (leafContexts.size()) {
+        case 0:
+          assert numDocs() == 0;
+          docs = new BitDocSet(BitDocSet.empty().getFixedBitSet(), 0);
+          break;
+        case 1:
+          final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+          final FixedBitSet fbs;
+          if (onlySegLiveDocs == null) {
+            final int onlySegMaxDoc = maxDoc();
+            fbs = new FixedBitSet(onlySegMaxDoc);
+            fbs.set(0, onlySegMaxDoc);
+          } else {
+            fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+          }
+          assert fbs.cardinality() == numDocs();
+          docs = new BitDocSet(fbs, numDocs());
+          break;
+        default:
+          final FixedBitSet bs = new FixedBitSet(maxDoc());
+          for (LeafReaderContext ctx : leafContexts) {
+            final LeafReader r = ctx.reader();
+            final Bits segLiveDocs = r.getLiveDocs();
+            final int segDocBase = ctx.docBase;
+            int segOrd = r.maxDoc() - 1;
+            if (segLiveDocs == null) {
+              do {
+                bs.set(segDocBase + segOrd);
+              } while (segOrd-- > 0);
+            } else {
+              do {
+                if (segLiveDocs.get(segOrd)) {
+                  bs.set(segDocBase + segOrd);

Review comment:
       I don't think the 2-arg version works _here_ because the two-arg version works on ranges. In order to call it I think you'd need more complex logic, scanning contiguous ranges and then calling set(int, int) when you encounter boundaries (deleted docs) ... I'm thinking that's not worth the effort here?




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r801238955



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -865,9 +929,31 @@ public BitDocSet getLiveDocSet() throws IOException {
     // Going through the filter cache will provide thread safety here if we only had getLiveDocs,
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
-    if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+    if (docs != null) {
+      matchAllDocsCacheHitCount.incrementAndGet();
+    } else {
+      if (matchAllDocsCacheComputationTracker.compareAndSet(Long.MIN_VALUE, 0)) {
+        // run the initial/only/final future inline
+        // This thread will block execution here and `liveDocsFuture.get()` (below) should then return immediately
+        liveDocsFuture.run();
+      } else {
+        // another thread has already called `computeLiveDocs.run()`; this thread will block on
+        // `liveDocsFuture.get()` (below)
+        if (matchAllDocsCacheComputationTracker.getAndIncrement() < 0) {

Review comment:
       :+1: ... any reasons I might have had for doing it this way are obviated by following your suggestion below, which just leans on a single-element CaffeineCache to handle the concurrency issues. 




-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r804061361



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       was that uncovered by a test (which)?




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808362089



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -119,10 +128,18 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /**
+   * Special-case cache to handle the lazy-init of {@link #liveDocs}.
+   */
+  private final SolrCache<MatchAllDocsQuery,BitDocSet> liveDocsCache;

Review comment:
       I don't feel strongly about it, and would be satisfied with an additional line in the comment saying why it is what it is.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799560079



##########
File path: solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java
##########
@@ -98,7 +102,15 @@ public void testCacheAssumptions() throws Exception {
 
     // This gets 0 docs back. Use 10000 instead of 1 for timeAllowed and it gets 100 back and the for loop below
     // succeeds.
-    String response = JQ(req("q", "*:*", "fq", fq, "indent", "true", "timeAllowed", "1", "sleep", sleep));
+    // TODO: address crosstalk between test methods; failures here can be triggered by cache consultation

Review comment:
       Yes, even more so really, since the special-casing for MatchAllDocs currently just throws it to filterCache. Now that I write this, it occurs to me that if special-casing matchAllDocs maybe we can special-case it to _bypass_ the filterCache entirely. I'll give that a try ...




-- 
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@solr.apache.org

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



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


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r800033599



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -865,9 +936,26 @@ public BitDocSet getLiveDocSet() throws IOException {
     // Going through the filter cache will provide thread safety here if we only had getLiveDocs,
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
-    if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+    if (docs != null) {
+      matchAllDocsCacheHitCount.incrementAndGet();
+    } else {
+      synchronized (matchAllDocsCacheComputationTracker) {
+        if (liveDocsFuture != null) {
+          // use future if it already exists
+          assert matchAllDocsCacheComputationTracker.incrementAndGet() > 0;
+        } else {
+          // otherwise create the initial/only future, and run it inline
+          assert matchAllDocsCacheComputationTracker.getAndSet(0) == Long.MIN_VALUE;
+          liveDocsFuture = new FutureTask<>(this::computeLiveDocs);
+          liveDocsFuture.run(); // first caller will block execution here
+        }
+      }
+      try {
+        docs = liveDocsFuture.get(); // subsequent callers block here, waiting for initial/only execution to complete

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `SolrIndexSearcher.getLiveDocSet()` reads without synchronization from `this.liveDocsFuture`. Potentially races with write in method `SolrIndexSearcher.getDocSetNC(...)`.
    Reporting because this access may occur on a background thread.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r801238310



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       :+1: done in dfff5f792a52ff4c8ed87d4495745340c7af10e6. Lmk if this is _too_ exhaustive, or if I've missed anything? 




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r806048246



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,20 +880,92 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set multiple times,
+   * but should always be set to the same value, as all set values should pass through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+  private final IOFunction<MatchAllDocsQuery, BitDocSet> computeLiveDocs = this::computeLiveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs(Query q) {
+    assert q == MATCH_ALL_DOCS_QUERY;
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            copyTo(segLiveDocs, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private static void copyTo(Bits segLiveDocs, int sourceMaxDoc, FixedBitSet bs, int segDocBase) {

Review comment:
       This is the kind of method that seems like it really wants unit tests. I spent quite a bit of time, and I think your logic is all correct, but I know I would be terrified to touch it in the future.

##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -45,6 +46,39 @@ public static boolean isNegative(Query q) {
     return true;
   }
 
+  /**
+   * Recursively unwraps the specified query to determine whether it is capable of producing a score
+   * that varies across different documents. Returns true if this query is not capable of producing a
+   * varying score (i.e., it is a constant score query).
+   */
+  public static boolean isConstantScoreQuery(Query q) {

Review comment:
       This whole method makes me want to add an isScoring method to lucene Query

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -883,16 +981,30 @@ public Bits getLiveDocsBits() throws IOException {
     return getIndexReader().hasDeletions() ? getLiveDocSet().getBits() : null;
   }
 
-  /** @lucene.internal */
-  public boolean isLiveDocsInstantiated() {
-    return liveDocs != null;
-  }
-
-  /** @lucene.internal */
-  public void setLiveDocs(DocSet docs) {
-    // a few places currently expect BitDocSet
-    assert docs.size() == numDocs();
-    this.liveDocs = makeBitDocSet(docs);
+  /**
+   * If some process external to {@link SolrIndexSearcher} has produced a DocSet whose cardinality matches
+   * that of `liveDocs`, this method provides such caller the ability to offer its own DocSet to be cached
+   * in the searcher. The caller should then use the returned value (which may or may not be derived from
+   * the DocSet instance supplied), allowing more efficient memory use.
+   * @lucene.internal
+   */
+  public BitDocSet offerLiveDocs(Supplier<DocSet> docSetSupplier, int suppliedSize) {

Review comment:
       very clear javadoc, this makes sense, thank you!
   
   We don't currently have the convention to do so, but what do you think about annotating this with `com.google.errorprone.annotations.@CheckReturnValue` so that it would cause compilation error if the return value was ignored?

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");

Review comment:
       use the constants in SolrCache instead of strings here?

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -119,10 +128,18 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /**
+   * Special-case cache to handle the lazy-init of {@link #liveDocs}.
+   */
+  private final SolrCache<MatchAllDocsQuery,BitDocSet> liveDocsCache;

Review comment:
       Using a CaffeineCache for a single element here seems heavy. Would it be simpler to use a single CompletableFuture? Or do we take advantage of the stats on the cache here? I guess that doesn't work with offerLiveDocs?

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,20 +880,92 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set multiple times,
+   * but should always be set to the same value, as all set values should pass through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+  private final IOFunction<MatchAllDocsQuery, BitDocSet> computeLiveDocs = this::computeLiveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs(Query q) {
+    assert q == MATCH_ALL_DOCS_QUERY;

Review comment:
       This is a strange signature/assertion. Why not do `computeLiveDocs(MatchAllDocsQuery q)`? Actually, the query isn't even used at all, so why does it matter?

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }

Review comment:
       The test framework takes care of this for us already, but it's fine to leave it here.

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...

Review comment:
       What does this mean? Did you want to do an exists query that happens to match all docs, like `str:*` (or `id:*`)

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.

Review comment:
       What's the issue here? I'm not entirely sure what I'm looking for.

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors

Review comment:
       `assertNull` here

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");
+    long naiveHits = coreToLiveDocsNaiveCacheHitCount(core);
+
+    assertEquals(1, inserts);
+    assertEquals(nThreads - 1, hits + naiveHits);
+    assertTrue(asyncHits <= hits);
+
+    // NOTE: The assertion below is commented out because, although it may _often_ be true, it is dependent
+    // on timing/thread scheduling; in practice it happens that not infrequently `asyncHits == 0` (e.g., if matchAllDocs
+    // computation happens quickly, and/or if subsequent threads were delayed).
+    //
+    // It seems that the assertion below more frequently succeeds when this test is run in isolation; e.g.:
+    // `gradlew :solr:core:test --tests "org.apache.solr.search.TestMainQueryCaching.testConcurrentMatchAllDocsInitialization"`
+
+    //assertTrue("expected asyncHits > 0; found asyncHits=" + asyncHits, asyncHits > 0);

Review comment:
       We could introduce delay at some point to cause threads to stall via TestInjection, but I'm fine without it.

##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -45,6 +46,39 @@ public static boolean isNegative(Query q) {
     return true;
   }
 
+  /**
+   * Recursively unwraps the specified query to determine whether it is capable of producing a score
+   * that varies across different documents. Returns true if this query is not capable of producing a
+   * varying score (i.e., it is a constant score query).
+   */
+  public static boolean isConstantScoreQuery(Query q) {
+    for (;;) {
+      if (q instanceof BoostQuery) {
+        q = ((BoostQuery) q).getQuery();
+      } else if (q instanceof WrappedQuery) {
+        q = ((WrappedQuery) q).getWrappedQuery();
+      } else if (q instanceof ConstantScoreQuery) {
+        return true;
+      } else if (q instanceof MatchAllDocsQuery) {
+        return true;
+      } else if (q instanceof MatchNoDocsQuery) {
+        return true;
+      } else if (q instanceof Filter || q instanceof SolrConstantScoreQuery) {
+        // TODO: this clause will be replaced with `q instanceof DocSetQuery`, pending SOLR-12336

Review comment:
       This has been merged :)

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");

Review comment:
       Yea, I can go either way. My thought was that if the constant ever moves, or the metric path moves, we'd want tests to fail compilation until they are updated. But this makes sense too. I'm not committed to either way.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838951458



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -118,10 +122,22 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /** Special-case cache to handle the lazy-init of {@link #liveDocs}. */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private final CompletableFuture<BitDocSet>[] liveDocsCache = new CompletableFuture[1];

Review comment:
       This thread should also be addressed by b3b38a1b2e280febd6a4a86464fad36bea4c56b8




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808357146



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,20 +880,92 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set multiple times,
+   * but should always be set to the same value, as all set values should pass through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+  private final IOFunction<MatchAllDocsQuery, BitDocSet> computeLiveDocs = this::computeLiveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs(Query q) {
+    assert q == MATCH_ALL_DOCS_QUERY;
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            copyTo(segLiveDocs, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private static void copyTo(Bits segLiveDocs, int sourceMaxDoc, FixedBitSet bs, int segDocBase) {

Review comment:
       Absolutely, thanks for raising that point. I'll make sure to add something before committing this.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808365564



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,20 +880,92 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set multiple times,
+   * but should always be set to the same value, as all set values should pass through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+  private final IOFunction<MatchAllDocsQuery, BitDocSet> computeLiveDocs = this::computeLiveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs(Query q) {
+    assert q == MATCH_ALL_DOCS_QUERY;

Review comment:
       I guess the thought was that this method looks tempting to call ... "computeLiveDocs? Sure I wanna do that!". It's really supposed to be very locked-down in terms of the contexts from which it's called (and even then not called directly -- it's called via CaffeinCache.computeIfAbsent).
   
   Reverse-engineering my subconscious thought process here: normally `private` access modifier alone would take care of that, but in a class as sprawling as SIS the power/semantics of `private` risk being diluted. Could change the method signature to MatchAllDocsQuery, yes; but paradoxically that might make it seem _more_ like it can be called by anyone with a MatchAllDocsQuery.
   
   If this logic makes sense I'll add a comment explaining this in a more succinct way. I don't mind removing the assertion, if you prefer.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r802084829



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       `org.apache.solr.search.QueryUtils#combineQueryAndFilter` wraps a ConstantScoringQuery but it could avoid it.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1034980744


   Interesting stuff going on over in LUCENE-10418 as well that maybe this could compliment in the future?


-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r800100152



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,8 +868,58 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
+  private static final Query MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
   private volatile BitDocSet liveDocs;
+  private final FutureTask<BitDocSet> liveDocsFuture = new FutureTask<>(this::computeLiveDocs);
+
+  private BitDocSet computeLiveDocs() {

Review comment:
       So you wrote all this low level code to avoid a simple MatchAllDocsQuery with a DocSetCollector? See org.apache.solr.search.SolrIndexSearcher.getDocSetNC which calls DocSetUtil.  Even if we choose to keep your low level code, I'd propose it go to a special case inside of DocSetUtil when the query is MatchAllDocsQuery.

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       I've seen a need for this elsewhere.  Let's make it handle other cases too (e.g. MatchAllDocsQuery, MatchNoDocsQuery, "Filter" (soon to be DocSetQuery)) and be recursive to the cases below (hey why not).  This could move to QueryUtils rather than having the massive SolrIndexSearcher grow

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -865,9 +929,31 @@ public BitDocSet getLiveDocSet() throws IOException {
     // Going through the filter cache will provide thread safety here if we only had getLiveDocs,
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
-    if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+    if (docs != null) {
+      matchAllDocsCacheHitCount.incrementAndGet();
+    } else {
+      if (matchAllDocsCacheComputationTracker.compareAndSet(Long.MIN_VALUE, 0)) {
+        // run the initial/only/final future inline
+        // This thread will block execution here and `liveDocsFuture.get()` (below) should then return immediately
+        liveDocsFuture.run();
+      } else {
+        // another thread has already called `computeLiveDocs.run()`; this thread will block on
+        // `liveDocsFuture.get()` (below)
+        if (matchAllDocsCacheComputationTracker.getAndIncrement() < 0) {

Review comment:
       The Future + AtomicLong thing looks overly complicated; maybe I'm not getting why it needs to be so complicated.  Can we have a simple double-check lock with synchronized to ensure only one creation happens?  We've all seen this pattern many times, I'm sure.  We don't even need the liveDocs to be volatile since it's an immutable object, not unlike a String, except for the lazy populated size but that doesn't matter as it's idempotent.

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -2281,6 +2455,11 @@ public void initializeMetrics(SolrMetricsContext parentContext, String scope) {
     parentContext.gauge(() -> openTime, true, "openedAt", Category.SEARCHER.toString(), scope);
     parentContext.gauge(() -> warmupTime, true, "warmupTime", Category.SEARCHER.toString(), scope);
     parentContext.gauge(() -> registerTime, true, "registeredAt", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(fullSortCount::get, true, "fullSortCount", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(skipSortCount::get, true, "skipSortCount", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(matchAllDocsCacheConsultationCount::get, true, "matchAllDocsCacheConsultationCount", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(matchAllDocsCacheHitCount::get, true, "matchAllDocsCacheHitCount", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(matchAllDocsCacheComputationTracker::get, true, "matchAllDocsCacheComputationTracker", Category.SEARCHER.toString(), scope);

Review comment:
       Is there value in publishing that "Tracker"?  It doesn't even sound like a metric.
   
   If we want these metrics, perhaps we should instead have a one-element Cache, which already publishes metrics and in a standard way across other Caches.  This is just an idea... if it's not easy then nevermind.
   

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,8 +868,58 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
+  private static final Query MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
   private volatile BitDocSet liveDocs;
+  private final FutureTask<BitDocSet> liveDocsFuture = new FutureTask<>(this::computeLiveDocs);
+
+  private BitDocSet computeLiveDocs() {
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return new BitDocSet(DocSet.empty().getFixedBitSet(), 0);

Review comment:
       This looks round-about.  Just return `new FixedBitSet(0);`




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799557771



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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 java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  static int NUM_DOCS = 100;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < NUM_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(NUM_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    assertU(commit());
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  @Test
+  public void testQueryCaching() throws Exception {
+    String q = "str:d*";
+    String constQ = "("+q+")^=1.0"; // wrapped as a ConstantScoreQuery
+
+    for (int i = 0; i < 6; i++) {

Review comment:
       :+1: I like the latter idea, I'll give that a shot.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r804247030



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       coincidentally enough I think this may be somewhat related to your comment on [PR #529](https://github.com/apache/solr/pull/529#discussion_r803130966)? Given that this issue has some TODOs related to SOLR-12336 anyway, I'll probably just wait to merge this until SOLR-12336 lands on main and 9_x.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799561070



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1405,14 +1446,32 @@ private void getDocListC(QueryResult qr, QueryCommand cmd) throws IOException {
         out.docSet = getDocSet(cmd.getQuery(), cmd.getFilter());
         List<Query> filterList = cmd.getFilterList();
         if (filterList != null && !filterList.isEmpty()) {
-          out.docSet = out.docSet.intersection(getDocSet(cmd.getFilterList()));
+          out.docSet = DocSetUtil.getDocSet(out.docSet.intersection(getDocSet(filterList)), this);
         }
       }
       // todo: there could be a sortDocSet that could take a list of
       // the filters instead of anding them first...
       // perhaps there should be a multi-docset-iterator
-      sortDocSet(qr, cmd);
+      if (needSort) {
+        fullSortCount++;
+        sortDocSet(qr, cmd);
+      } else {
+        skipSortCount++;
+        // put unsorted list in place
+        out.docList = constantScoreDocList(cmd.getOffset(), cmd.getLen(), out.docSet);
+        if (0 == cmd.getSupersetMaxDoc()) {
+          // this is the only case where `cursorMark && !needSort`
+          qr.setNextCursorMark(cmd.getCursorMark());

Review comment:
       Ahh good catch. Will add one, and for `rows=1` (which should _not_ hit the sort optimization)




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799513478



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1302,41 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {
+    if (q instanceof BoostQuery) {
+      // ConstantScoreQueries are often (always?) wrapped in BoostQuery to assign specific score
+      q = ((BoostQuery)q).getQuery();
+    }
+    return q instanceof ConstantScoreQuery;
+  }
+
+  private static boolean sortIncludesOtherThanScore(final Sort sort) {
+    if (sort == null) {
+      return false;
+    }
+    final SortField[] sortFields = sort.getSort();
+    return sortFields.length > 1 || sortFields[0].getType() != Type.SCORE;
+  }
+
+  private boolean useFilterCacheForDynamicScoreQuery(boolean needSort, QueryCommand cmd) {

Review comment:
       👍 

##########
File path: solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java
##########
@@ -98,7 +102,15 @@ public void testCacheAssumptions() throws Exception {
 
     // This gets 0 docs back. Use 10000 instead of 1 for timeAllowed and it gets 100 back and the for loop below
     // succeeds.
-    String response = JQ(req("q", "*:*", "fq", fq, "indent", "true", "timeAllowed", "1", "sleep", sleep));
+    // TODO: address crosstalk between test methods; failures here can be triggered by cache consultation

Review comment:
       Is this still an issue after you added the special MatchAllDocs cases?

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1302,41 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {
+    if (q instanceof BoostQuery) {
+      // ConstantScoreQueries are often (always?) wrapped in BoostQuery to assign specific score

Review comment:
       I suspect this partially goes away after #529

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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 java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  static int NUM_DOCS = 100;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < NUM_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(NUM_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    assertU(commit());
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  @Test
+  public void testQueryCaching() throws Exception {
+    String q = "str:d*";
+    String constQ = "("+q+")^=1.0"; // wrapped as a ConstantScoreQuery
+
+    for (int i = 0; i < 6; i++) {

Review comment:
       An interesting idea I just had was to randomize the order that we run all of these cases. It _shouldn't_ matter because you reload in between but who knows.
   
   The other idea would be to split all of these out into separate test methods, and give them a common assertMetricCounts helper method or something like that.

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1405,14 +1446,32 @@ private void getDocListC(QueryResult qr, QueryCommand cmd) throws IOException {
         out.docSet = getDocSet(cmd.getQuery(), cmd.getFilter());
         List<Query> filterList = cmd.getFilterList();
         if (filterList != null && !filterList.isEmpty()) {
-          out.docSet = out.docSet.intersection(getDocSet(cmd.getFilterList()));
+          out.docSet = DocSetUtil.getDocSet(out.docSet.intersection(getDocSet(filterList)), this);
         }
       }
       // todo: there could be a sortDocSet that could take a list of
       // the filters instead of anding them first...
       // perhaps there should be a multi-docset-iterator
-      sortDocSet(qr, cmd);
+      if (needSort) {
+        fullSortCount++;
+        sortDocSet(qr, cmd);
+      } else {
+        skipSortCount++;
+        // put unsorted list in place
+        out.docList = constantScoreDocList(cmd.getOffset(), cmd.getLen(), out.docSet);
+        if (0 == cmd.getSupersetMaxDoc()) {
+          // this is the only case where `cursorMark && !needSort`
+          qr.setNextCursorMark(cmd.getCursorMark());

Review comment:
       Is there a test that tries to do `cursorMark=*&rows=0`




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1083391036


   I'm hoping to commit this this week.
   
   I did want first to specifically call out that _after_ @dsmiley's PR approval, there were changes to the way the liveDocs DocSet is cached: basically, no longer uses a 1-element CaffeineCache and instead handles synchronization manually within SIS, adding a few metrics mainly to support test assertions. I'm comfortable with this approach and pretty sure the concurrency stuff is correct and idiomatic, but in case anyone wants to take another look before I commit, the substance of the new approach is mostly at [SIS.populateLiveDocs](https://github.com/apache/solr/blob/91222454725122a76399348eb3143e373d2b222c/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java#L1007-L1035).
   
   Thanks again!


-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838891491



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -118,10 +122,22 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /** Special-case cache to handle the lazy-init of {@link #liveDocs}. */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private final CompletableFuture<BitDocSet>[] liveDocsCache = new CompletableFuture[1];

Review comment:
       You can do with only 1 suppression if you use `Array.newInstance` - there's an example in ZkController




-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838794953



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -118,10 +122,22 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /** Special-case cache to handle the lazy-init of {@link #liveDocs}. */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private final CompletableFuture<BitDocSet>[] liveDocsCache = new CompletableFuture[1];

Review comment:
       I would declare it to be simply Future and not specifically CompletableFuture as it's an implementation detail that can be local to the only method that uses it.
   
   BTW is any of that SuppressWarnings actually appropriate here?  I don't see either occuring.  There was a big effort to scrub the codebase of needless suppressions and also declaring things better so that we needn't have a warning in the first place.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1030362953


   @madrob thanks very much for the review! I believe I've addressed all the concerns you raised. The only one I feel certain might merit more attention is the question of MatchAllDocs special-case, which I've adapted to now not (ever?) hit the filterCache, in 1e6cfeffbabd400b37afe8af8c1bc29d45d8de79. That allowed to revert some of the test workarounds (including a preexisting workaround related to pure-negative filters). Keeping MatchAllDocs off the `filterCache` is I think more generally a Good Thing -- in fact the change addressed an [existing TODO comment](https://github.com/apache/solr/blob/cb2e58f6569704c45387a8d094625ba8e014776b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java#L869).


-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799881581



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+      switch (leafContexts.size()) {
+        case 0:
+          assert numDocs() == 0;
+          docs = new BitDocSet(BitDocSet.empty().getFixedBitSet(), 0);
+          break;
+        case 1:
+          final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+          final FixedBitSet fbs;
+          if (onlySegLiveDocs == null) {
+            final int onlySegMaxDoc = maxDoc();
+            fbs = new FixedBitSet(onlySegMaxDoc);
+            fbs.set(0, onlySegMaxDoc);
+          } else {
+            fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+          }
+          assert fbs.cardinality() == numDocs();
+          docs = new BitDocSet(fbs, numDocs());
+          break;
+        default:
+          final FixedBitSet bs = new FixedBitSet(maxDoc());
+          for (LeafReaderContext ctx : leafContexts) {
+            final LeafReader r = ctx.reader();
+            final Bits segLiveDocs = r.getLiveDocs();
+            final int segDocBase = ctx.docBase;
+            int segOrd = r.maxDoc() - 1;
+            if (segLiveDocs == null) {
+              do {
+                bs.set(segDocBase + segOrd);
+              } while (segOrd-- > 0);
+            } else {
+              do {
+                if (segLiveDocs.get(segOrd)) {
+                  bs.set(segDocBase + segOrd);

Review comment:
       I don't think the 2-arg version works _here_ because the two-arg version works on ranges. In order to call it I think you'd need more complex logic, scanning contiguous ranges and then calling set(int, int) when you encounter boundaries (deleted docs) ... I'm thinking that's not worth the effort here?
   
   EDIT: nevermind my initial reluctance; I went ahead with the range-scanning and bulk-set calls and the logic ended up being pretty straightforward.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r801883824



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       >I've seen a need for this elsewhere
   
   I looked for other places in the codebase where this could be used, and I didn't immediately find anything. Perhaps I missed something? It does indeed seem general-purpose useful (even if only otherwise leveraged from 3rd-party plugins).




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799884140



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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 java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {

Review comment:
       Yes, checking for `WrappedQuery` wouldn't hurt (would need to respect the cache directives in ExtendedQueryBase then, which would be nice).




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799882766



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {

Review comment:
       You're right that this case kind of undermines (for this special case) the improvements you introduced with SOLR-15555 ... I'll see what I can do about that...




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808381274



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");
+    long naiveHits = coreToLiveDocsNaiveCacheHitCount(core);
+
+    assertEquals(1, inserts);
+    assertEquals(nThreads - 1, hits + naiveHits);
+    assertTrue(asyncHits <= hits);
+
+    // NOTE: The assertion below is commented out because, although it may _often_ be true, it is dependent
+    // on timing/thread scheduling; in practice it happens that not infrequently `asyncHits == 0` (e.g., if matchAllDocs
+    // computation happens quickly, and/or if subsequent threads were delayed).
+    //
+    // It seems that the assertion below more frequently succeeds when this test is run in isolation; e.g.:
+    // `gradlew :solr:core:test --tests "org.apache.solr.search.TestMainQueryCaching.testConcurrentMatchAllDocsInitialization"`
+
+    //assertTrue("expected asyncHits > 0; found asyncHits=" + asyncHits, asyncHits > 0);

Review comment:
       That'd be really cool. I actually kind of knocked myself out a little trying to do something like that but didn't want to reinvent the wheel or end up with something that needlessly slows down the test suite. If you know of a good (dare I say "right"?) way to do this I'm definitely interested.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808455051



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,20 +880,92 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set multiple times,
+   * but should always be set to the same value, as all set values should pass through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+  private final IOFunction<MatchAllDocsQuery, BitDocSet> computeLiveDocs = this::computeLiveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs(Query q) {
+    assert q == MATCH_ALL_DOCS_QUERY;

Review comment:
       obviated by simplifications related to 14f7ac5434626e5d3a068b8c2489b0d604171daa




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1029698056


   Added a new test suite (2 tests) that verifies the new sort-skipping optimization, and also folds in explicit tests for the useFilterForSortedQuery behavior (as newly updated), which wasn't really directly covered by existing tests.
   
   Pending further feedback, the only thing I _know_ needs to happen is change the "fix version" under which the CHANGES.txt entry is registered.


-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r798151164



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -2040,19 +2098,7 @@ protected void sortDocSet(QueryResult qr, QueryCommand cmd) throws IOException {
       leafCollector.collect(doc - base);

Review comment:
       @sonatype-lift ignore
   This is actually a revert, so unrelated to this PR; in any event I trust this will not manifest in practice as an NPE




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799196159



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1384,16 +1385,24 @@ private void getDocListC(QueryResult qr, QueryCommand cmd) throws IOException {
     // - we don't want score returned.
 
     // check if we should try and use the filter cache
-    boolean useFilterCache = false;
-    if ((flags & (GET_SCORES | NO_CHECK_FILTERCACHE)) == 0 && useFilterForSortedQuery && cmd.getSort() != null
-        && filterCache != null) {
-      useFilterCache = true;
-      SortField[] sfields = cmd.getSort().getSort();
-      for (SortField sf : sfields) {
-        if (sf.getType() == SortField.Type.SCORE) {
-          useFilterCache = false;
-          break;
-        }
+    final boolean needSort;
+    final boolean useFilterCache;
+    if ((flags & (GET_SCORES | NO_CHECK_FILTERCACHE)) != 0 || filterCache == null) {
+      needSort = true; // this value should be irrelevant when `useFilterCache=false`
+      useFilterCache = false;
+    } else {
+      final Sort sort;
+      if (q instanceof  MatchAllDocsQuery // special-case MatchAllDocsQuery: implicit default useFilterForSortedQuery=true
+              || (useFilterForSortedQuery && q instanceof ConstantScoreQuery)) { // default behavior should not risk filterCache thrashing
+        final SortField[] sortFields;
+        // We only need to sort if we're returning results AND sorting by something other than SCORE (sort by
+        // "score" alone is pointless for these constant score queries)
+        needSort = cmd.getLen() > 0 && (sort = cmd.getSort()) != null && ((sortFields = sort.getSort()).length > 1 || sortFields[0].getType() != Type.SCORE);

Review comment:
       :+1: ... refactored this (and a couple of other) complex 1-liner conditionals into methods with more transparent names.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r801241165



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {

Review comment:
       :+1: this should be addressed now, newly by lazy-initing `liveDocSet` via a single-element special-case CaffeineCache. 




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1034435148


   Thanks for the reviews, @madrob and @dsmiley! Pending further feedback, I hope to fix up the CHANGES.txt entry and commit to main tomorrow. Any thoughts on whether fix version 9.0 or 9.1?


-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1034442900


   It's internal and not particularly impactful to any plugin writer so I suggest 9.1.


-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r802990106



##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -54,23 +54,7 @@ public static boolean isNegative(Query q) {
    * varying score (i.e., it is a constant score query).
    */
   public static boolean isConstantScoreQuery(Query q) {
-    return isConstantScoreQuery(q, null);
-  }
-
-  private static Map<Query, Void> lazyInitSeen(Map<Query, Void> seen, Query add) {
-    if (seen == null) {
-      seen = new IdentityHashMap<>();
-    }
-    seen.put(add, null);
-    return seen;
-  }
-
-  /**
-   * Returns true if the specified query is guaranteed to assign the same score to all docs; otherwise false
-   * @param q query to be evaluated
-   * @param seen used to detect possible loops in nested query input
-   */
-  private static boolean isConstantScoreQuery(Query q, Map<Query, Void> seen) {
+    Map<Query, Void> seen = null; // lazy-init; this will be unnecessary in many cases

Review comment:
       I'd remove these comments as well.  I've never seen this loop checking done before for Queries and I've never seen a Query try to prevent this.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1034068134


   re: inter-relationship between DocSetUtil and SolrIndexSearcher.
   
   I see the general issue you're concerned about, but if it's ok I'd like to leave `computeLiveDocs` stuff in SIS for this PR, and if/when work on refactoring stuff out as a separate issue? From my perspective, leaving `computeLiveDocs` in SIS for now really boils down to keeping the method `private`, with the goal of limiting the scopes from which that method is directly accessible -- or really I suppose more trying to ensure that there's exactly _one_ fundamental/canonical way to get liveDocSet, and that is directly via `SIS.getLiveDocSet()`.


-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799556562



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1302,41 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {
+    if (q instanceof BoostQuery) {
+      // ConstantScoreQueries are often (always?) wrapped in BoostQuery to assign specific score

Review comment:
       Something to keep an eye out for; I applied a patch for #529 and this wrapping still seems to happen. I think this is a side-effect of the way "ConstantScoreQuery" gets specified in queries (`(clause)^=n`) where `n` is the constant score, which is applied via wrapping in a BoostQuery. In any event it doesn't look like that changes with #529




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r802988483



##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -54,23 +54,7 @@ public static boolean isNegative(Query q) {
    * varying score (i.e., it is a constant score query).
    */
   public static boolean isConstantScoreQuery(Query q) {
-    return isConstantScoreQuery(q, null);
-  }
-
-  private static Map<Query, Void> lazyInitSeen(Map<Query, Void> seen, Query add) {
-    if (seen == null) {
-      seen = new IdentityHashMap<>();
-    }
-    seen.put(add, null);
-    return seen;
-  }
-
-  /**
-   * Returns true if the specified query is guaranteed to assign the same score to all docs; otherwise false
-   * @param q query to be evaluated
-   * @param seen used to detect possible loops in nested query input
-   */
-  private static boolean isConstantScoreQuery(Query q, Map<Query, Void> seen) {
+    Map<Query, Void> seen = null; // lazy-init; this will be unnecessary in many cases

Review comment:
       This comment is also relevant:
   ```
   // NOTE: Neither WrappedQuery class nor its inner query are final, so there is a risk of direct loops
   // TODO: Only the queries we explicitly check for in this method are relevant wrt detecting loops, and
   //  only `WrappedQuery` currently presents a risk in that respect; we may be able to avoid this risk
   //  by more tightly restricting the `WrappedQuery` API (e.g., making the get/set methods `final`)?
   ```




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r801237620



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,8 +868,58 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
+  private static final Query MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
   private volatile BitDocSet liveDocs;
+  private final FutureTask<BitDocSet> liveDocsFuture = new FutureTask<>(this::computeLiveDocs);
+
+  private BitDocSet computeLiveDocs() {

Review comment:
       wrt low-level, I realized we needed at least to _check_ for the MatchAllDocsQuery special case in SolrIndexSearcher (in order to siphon that case off before consulting filterCache) -- and having realized that, I basically just ran with [this suggestion/todo comment](https://github.com/apache/solr/commit/1e63b32731bedf108aaeeb5d0a04d671f5663102#diff-99978700f1c69d6a5f6c2190f89c98cfe20c441161db5a183ec002e15cb1be28R857).
   
   I'm ok with losing the special-case low-level code (though it basically makes liveDocSet computation into a glorified array copy -- though to be fair I haven't profiled it). Tangentially: it strikes me that some of this code could be useful in partially restoring cache (e.g. filterCache) entries per-segment sometime down the line ...
   
   That said, although I initially went ahead an moved the code to DocSetUtil as you suggested, I actually ended up moving it back, for reasons explained in the commit message for e368bdd7008894939276f3b94891979cdc4f88b7:
   ```
       In practice it felt like an awkward (and perhaps somewhat dangerous?)
       fit to have `computeLiveDocs` live in `DocSetUtil`. The potential for
       deadlock was real; and the `liveDocs` concept is really very intimately
       associated with the SolrIndexSearcher per se; putting its computation
       in DocSetUtil might tempt people to call it there, when they should
       really just get liveDocs via `SolrIndexSearcher.getLiveDocSet()`
   ```
   I'm curious to know what you think. I'm find to move it back into DocSetUtil if you think that's better (or for that matter remove it entirely if you prefer).




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1030383561


   Yea, I'm super excited about this, it's starting to look like a pretty nice boost to what we do!


-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r806141738



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");

Review comment:
       FWIW I prefer readability in tests (String literals are plainly obvious).  And Yonik once pointed out that avoiding such constants in tests helps ensure we don't change something without seeing the impact (e.g. a client may depend on it).




-- 
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@solr.apache.org

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



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


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r798035783



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -2040,19 +2098,7 @@ protected void sortDocSet(QueryResult qr, QueryCommand cmd) throws IOException {
       leafCollector.collect(doc - base);

Review comment:
       *NULL_DEREFERENCE:*  object `leafCollector` last assigned on line 2088 could be null and is dereferenced at line 2098.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808379629



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");

Review comment:
       Interesting; I'm really ambivalent :-| ... I can see good arguments both ways. I'm inclined to let inertia win and leave it as-is?




-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838791801



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -935,22 +956,96 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set
+   * multiple times, but should always be set to the same value, as all set values should pass
+   * through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs() {
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all
+            // bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            DocSetUtil.copyTo(segLiveDocs, 0, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private BitDocSet populateLiveDocs(Supplier<BitDocSet> liveDocsSupplier) {
+    final boolean computeInline;
+    final CompletableFuture<BitDocSet> liveDocsCacheInstance;
+    synchronized (liveDocsCache) {
+      if (liveDocsCache[0] != null) {
+        computeInline = false;
+        liveDocsCacheInstance = liveDocsCache[0];
+      } else {
+        computeInline = true;
+        liveDocsCacheInstance = new CompletableFuture<>();
+        liveDocsCache[0] = liveDocsCacheInstance;
+      }
+    }
+    final BitDocSet docs;
+    if (computeInline) {
+      docs = liveDocsSupplier.get();
+      liveDocsCacheInstance.complete(docs);
+      liveDocs = docs;
+      liveDocsInsertsCount.increment();
+    } else {

Review comment:
       Why not do this in the synchronized block above?  It's natural that it would go there because we're initializing it -- a standard pattern to use synchronized in this way.  Yes it would cause callers to block (contention) but it's fair/expected.  They will any way, waiting on the CompletableFuture.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799880606



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+      switch (leafContexts.size()) {
+        case 0:
+          assert numDocs() == 0;
+          docs = new BitDocSet(BitDocSet.empty().getFixedBitSet(), 0);
+          break;
+        case 1:
+          final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+          final FixedBitSet fbs;
+          if (onlySegLiveDocs == null) {
+            final int onlySegMaxDoc = maxDoc();
+            fbs = new FixedBitSet(onlySegMaxDoc);
+            fbs.set(0, onlySegMaxDoc);
+          } else {
+            fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+          }
+          assert fbs.cardinality() == numDocs();
+          docs = new BitDocSet(fbs, numDocs());
+          break;
+        default:
+          final FixedBitSet bs = new FixedBitSet(maxDoc());
+          for (LeafReaderContext ctx : leafContexts) {
+            final LeafReader r = ctx.reader();
+            final Bits segLiveDocs = r.getLiveDocs();
+            final int segDocBase = ctx.docBase;
+            int segOrd = r.maxDoc() - 1;
+            if (segLiveDocs == null) {
+              do {
+                bs.set(segDocBase + segOrd);

Review comment:
       :+1: ha! yeah, not sure why it occurred to me above, but not here.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808458486



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...

Review comment:
       added a clarifying comment in b494419ca1eff547a73ecf81a608cb6abaa789a3 




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808356737



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -119,10 +128,18 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /**
+   * Special-case cache to handle the lazy-init of {@link #liveDocs}.
+   */
+  private final SolrCache<MatchAllDocsQuery,BitDocSet> liveDocsCache;

Review comment:
       Yeah; it started out as something ostensibly "simpler" like that, but once you layer in the metrics (mainly to support tests) it becomes less simple. I wonder how succinct it could be, maybe worth a shot.
   
   I'll mull this; if it's something you feel (somewhat) strongly about I'll take a crack at it.
   
   EDIT: nevermind, done in 14f7ac5434626e5d3a068b8c2489b0d604171daa -- were you thinking something along these lines?




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1042317609


   Thanks for the feedback, @madrob. I think everything should be addressed now except for the [unit test](#discussion_r806048246) for `copyTo(...)`, which will take a bit more time but should be fairly cut-and-dry as far as what's required.


-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808373074



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...

Review comment:
       Yeah this is confusing. No we actually want `*:*` here I think, because we're testing that even when `*:*` is specified, if the user _explicitly_ requests that `score` be returned, that disables the optimizations we have otherwise. This is an edge-case-of-an-edge-case, and probably an antipattern (to explicitly request `score` for `*:*`). But one practical consequence of this for testing it's useful to be able to bypass the optimization.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r806048246



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,20 +880,92 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set multiple times,
+   * but should always be set to the same value, as all set values should pass through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+  private final IOFunction<MatchAllDocsQuery, BitDocSet> computeLiveDocs = this::computeLiveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs(Query q) {
+    assert q == MATCH_ALL_DOCS_QUERY;
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            copyTo(segLiveDocs, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private static void copyTo(Bits segLiveDocs, int sourceMaxDoc, FixedBitSet bs, int segDocBase) {

Review comment:
       This is the kind of method that seems like it really wants unit tests. I spent quite a bit of time, and I think your logic is all correct, but I know I would be terrified to touch it in the future.

##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -45,6 +46,39 @@ public static boolean isNegative(Query q) {
     return true;
   }
 
+  /**
+   * Recursively unwraps the specified query to determine whether it is capable of producing a score
+   * that varies across different documents. Returns true if this query is not capable of producing a
+   * varying score (i.e., it is a constant score query).
+   */
+  public static boolean isConstantScoreQuery(Query q) {

Review comment:
       This whole method makes me want to add an isScoring method to lucene Query

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -883,16 +981,30 @@ public Bits getLiveDocsBits() throws IOException {
     return getIndexReader().hasDeletions() ? getLiveDocSet().getBits() : null;
   }
 
-  /** @lucene.internal */
-  public boolean isLiveDocsInstantiated() {
-    return liveDocs != null;
-  }
-
-  /** @lucene.internal */
-  public void setLiveDocs(DocSet docs) {
-    // a few places currently expect BitDocSet
-    assert docs.size() == numDocs();
-    this.liveDocs = makeBitDocSet(docs);
+  /**
+   * If some process external to {@link SolrIndexSearcher} has produced a DocSet whose cardinality matches
+   * that of `liveDocs`, this method provides such caller the ability to offer its own DocSet to be cached
+   * in the searcher. The caller should then use the returned value (which may or may not be derived from
+   * the DocSet instance supplied), allowing more efficient memory use.
+   * @lucene.internal
+   */
+  public BitDocSet offerLiveDocs(Supplier<DocSet> docSetSupplier, int suppliedSize) {

Review comment:
       very clear javadoc, this makes sense, thank you!
   
   We don't currently have the convention to do so, but what do you think about annotating this with `com.google.errorprone.annotations.@CheckReturnValue` so that it would cause compilation error if the return value was ignored?

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");

Review comment:
       use the constants in SolrCache instead of strings here?

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -119,10 +128,18 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /**
+   * Special-case cache to handle the lazy-init of {@link #liveDocs}.
+   */
+  private final SolrCache<MatchAllDocsQuery,BitDocSet> liveDocsCache;

Review comment:
       Using a CaffeineCache for a single element here seems heavy. Would it be simpler to use a single CompletableFuture? Or do we take advantage of the stats on the cache here? I guess that doesn't work with offerLiveDocs?

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,20 +880,92 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set multiple times,
+   * but should always be set to the same value, as all set values should pass through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+  private final IOFunction<MatchAllDocsQuery, BitDocSet> computeLiveDocs = this::computeLiveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs(Query q) {
+    assert q == MATCH_ALL_DOCS_QUERY;

Review comment:
       This is a strange signature/assertion. Why not do `computeLiveDocs(MatchAllDocsQuery q)`? Actually, the query isn't even used at all, so why does it matter?

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }

Review comment:
       The test framework takes care of this for us already, but it's fine to leave it here.

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...

Review comment:
       What does this mean? Did you want to do an exists query that happens to match all docs, like `str:*` (or `id:*`)

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.

Review comment:
       What's the issue here? I'm not entirely sure what I'm looking for.

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors

Review comment:
       `assertNull` here

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");
+    long naiveHits = coreToLiveDocsNaiveCacheHitCount(core);
+
+    assertEquals(1, inserts);
+    assertEquals(nThreads - 1, hits + naiveHits);
+    assertTrue(asyncHits <= hits);
+
+    // NOTE: The assertion below is commented out because, although it may _often_ be true, it is dependent
+    // on timing/thread scheduling; in practice it happens that not infrequently `asyncHits == 0` (e.g., if matchAllDocs
+    // computation happens quickly, and/or if subsequent threads were delayed).
+    //
+    // It seems that the assertion below more frequently succeeds when this test is run in isolation; e.g.:
+    // `gradlew :solr:core:test --tests "org.apache.solr.search.TestMainQueryCaching.testConcurrentMatchAllDocsInitialization"`
+
+    //assertTrue("expected asyncHits > 0; found asyncHits=" + asyncHits, asyncHits > 0);

Review comment:
       We could introduce delay at some point to cause threads to stall via TestInjection, but I'm fine without it.

##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -45,6 +46,39 @@ public static boolean isNegative(Query q) {
     return true;
   }
 
+  /**
+   * Recursively unwraps the specified query to determine whether it is capable of producing a score
+   * that varies across different documents. Returns true if this query is not capable of producing a
+   * varying score (i.e., it is a constant score query).
+   */
+  public static boolean isConstantScoreQuery(Query q) {
+    for (;;) {
+      if (q instanceof BoostQuery) {
+        q = ((BoostQuery) q).getQuery();
+      } else if (q instanceof WrappedQuery) {
+        q = ((WrappedQuery) q).getWrappedQuery();
+      } else if (q instanceof ConstantScoreQuery) {
+        return true;
+      } else if (q instanceof MatchAllDocsQuery) {
+        return true;
+      } else if (q instanceof MatchNoDocsQuery) {
+        return true;
+      } else if (q instanceof Filter || q instanceof SolrConstantScoreQuery) {
+        // TODO: this clause will be replaced with `q instanceof DocSetQuery`, pending SOLR-12336

Review comment:
       This has been merged :)




-- 
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@solr.apache.org

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



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


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r798151192



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -2040,19 +2098,7 @@ protected void sortDocSet(QueryResult qr, QueryCommand cmd) throws IOException {
       leafCollector.collect(doc - base);

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1084725965


   Thanks, @dsmiley! I think ef4e35e065be8385a3ca7219df471d68bb567c51 addresses all these concerns.


-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r839472975



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -935,22 +956,83 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set
+   * multiple times, but should always be set to the same value, as all set values should pass
+   * through `liveDocsCache.computeIfAbsent`

Review comment:
       obsolete comment

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -118,10 +121,23 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /**
+   * Special-case cache, mainly used as a synchronization point to handle the lazy-init of {@link
+   * #liveDocs}.
+   */
+  private final BitDocSet[] liveDocsCache = new BitDocSet[1];

Review comment:
       This can replaced with `Object liveDocsSync = new Object()`; no?
   Furthermore, declare it adjacent to liveDocs; keep things together/organized.

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -935,22 +956,83 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set
+   * multiple times, but should always be set to the same value, as all set values should pass
+   * through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs() {
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all
+            // bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            DocSetUtil.copyTo(segLiveDocs, 0, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private BitDocSet populateLiveDocs(Supplier<BitDocSet> liveDocsSupplier) {
+    final BitDocSet inlineDocs;

Review comment:
       As I suggested in my comment above, this can be simplified with merely an Object used to synchronize.  No need for a "inlineDocs" local variable.  No need for an array or anything similar-is.  Simplicity!

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -968,19 +1050,23 @@ public Bits getLiveDocsBits() throws IOException {
   }
 
   /**
+   * If some process external to {@link SolrIndexSearcher} has produced a DocSet whose cardinality
+   * matches that of `liveDocs`, this method provides such caller the ability to offer its own
+   * DocSet to be cached in the searcher. The caller should then use the returned value (which may
+   * or may not be derived from the DocSet instance supplied), allowing more efficient memory use.
+   *
    * @lucene.internal
    */
-  public boolean isLiveDocsInstantiated() {
-    return liveDocs != null;
-  }
-
-  /**
-   * @lucene.internal
-   */
-  public void setLiveDocs(DocSet docs) {
+  public BitDocSet offerLiveDocs(Supplier<DocSet> docSetSupplier, int suppliedSize) {
+    assert suppliedSize == numDocs();
+    BitDocSet ret = liveDocs;
+    if (ret != null) {
+      liveDocsNaiveCacheHitCount.increment();
+      return ret;
+    }
     // a few places currently expect BitDocSet
-    assert docs.size() == numDocs();
-    this.liveDocs = makeBitDocSet(docs);
+    ret = populateLiveDocs(() -> makeBitDocSet(docSetSupplier.get()));

Review comment:
       pointless; I'd do this as one line -- just return it




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808369044



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.

Review comment:
       The goal was to allow explicitly specifying `-Psolr.test.useFilterForSortedQuery=true|false` on the command line, but it looks like the system properties are filtered by gradle maybe (?) ... in any event when I tried to set this property explicitly it didn't get picked up in the actual test. If there's a quick answer, I'd be curious to know; but either way I agree with your implicit suggestion that leaving this as a TODO comment doesn't make sense ...




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1042325715


   As a result of 14f7ac5434626e5d3a068b8c2489b0d604171daa, we could revert the addition of the `asyncHits` metric to `CaffeineCache.java`. I think it's a potentially useful metric? But Mike, David, unless you think it should stay I'm inclined to remove it from this PR (and tbh I probably won't pursue adding it without some renewed motivation to do so).


-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r812357947



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -854,20 +880,92 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set multiple times,
+   * but should always be set to the same value, as all set values should pass through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+  private final IOFunction<MatchAllDocsQuery, BitDocSet> computeLiveDocs = this::computeLiveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs(Query q) {
+    assert q == MATCH_ALL_DOCS_QUERY;
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            copyTo(segLiveDocs, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private static void copyTo(Bits segLiveDocs, int sourceMaxDoc, FixedBitSet bs, int segDocBase) {

Review comment:
       Ok, added a unit test for this (to TestDocSet.java) in 8defddff3e18906ac03f21ea85c91405c516397e.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r804016154



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       FYI, I had to revert the above change; see: c9ccf5c8be98a169742ffc924dda24aa8d3bc4e0




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r804074552



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       Yes (sorry, I mentioned it in the commit):
   ```
   gradlew :solr:core:test --tests "org.apache.solr.search.TestPseudoReturnFields.testMultiValued"
   ```




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838897149



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -935,22 +956,96 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set
+   * multiple times, but should always be set to the same value, as all set values should pass
+   * through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs() {
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all
+            // bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            DocSetUtil.copyTo(segLiveDocs, 0, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private BitDocSet populateLiveDocs(Supplier<BitDocSet> liveDocsSupplier) {
+    final boolean computeInline;
+    final CompletableFuture<BitDocSet> liveDocsCacheInstance;
+    synchronized (liveDocsCache) {
+      if (liveDocsCache[0] != null) {
+        computeInline = false;
+        liveDocsCacheInstance = liveDocsCache[0];
+      } else {
+        computeInline = true;
+        liveDocsCacheInstance = new CompletableFuture<>();
+        liveDocsCache[0] = liveDocsCacheInstance;
+      }
+    }
+    final BitDocSet docs;
+    if (computeInline) {
+      docs = liveDocsSupplier.get();
+      liveDocsCacheInstance.complete(docs);
+      liveDocs = docs;
+      liveDocsInsertsCount.increment();
+    } else {

Review comment:
       If we do everything in the synchronized block, then there's no reason for the CompletableFuture at all anymore, right? I feel like at that point we're losing concurrency gains, but would have to think about it more to be sure.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838950285



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -935,22 +956,96 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set
+   * multiple times, but should always be set to the same value, as all set values should pass
+   * through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs() {
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all
+            // bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            DocSetUtil.copyTo(segLiveDocs, 0, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private BitDocSet populateLiveDocs(Supplier<BitDocSet> liveDocsSupplier) {
+    final boolean computeInline;
+    final CompletableFuture<BitDocSet> liveDocsCacheInstance;
+    synchronized (liveDocsCache) {
+      if (liveDocsCache[0] != null) {
+        computeInline = false;
+        liveDocsCacheInstance = liveDocsCache[0];
+      } else {
+        computeInline = true;
+        liveDocsCacheInstance = new CompletableFuture<>();
+        liveDocsCache[0] = liveDocsCacheInstance;
+      }
+    }
+    final BitDocSet docs;
+    if (computeInline) {
+      docs = liveDocsSupplier.get();
+      liveDocsCacheInstance.complete(docs);
+      liveDocs = docs;
+      liveDocsInsertsCount.increment();
+    } else {

Review comment:
       I think you're right. And I think you don't even lose on concurrency gains:
   
   In both cases execution blocks until the result is available. It doesn't matter whether you're blocking for a synchronized lock or on getting results from the Future. The only way it matters is if you're attempting to make a distinction between "I'm waiting because a computation is in progress" and "I had to get a lock on the object that will ensure computation has completed". So if we lose the "asyncHits" metric (I'm fine with that), then things can be much simpler.
   
   Tried this out with b3b38a1b2e280febd6a4a86464fad36bea4c56b8 ... seems good to me?




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838851191



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -935,22 +956,96 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
     return filterCache.computeIfAbsent(query, q -> getDocSetNC(q, null));
   }
 
-  private static Query matchAllDocsQuery = new MatchAllDocsQuery();
-  private volatile BitDocSet liveDocs;
+  private static final MatchAllDocsQuery MATCH_ALL_DOCS_QUERY = new MatchAllDocsQuery();
+
+  /**
+   * A naively cached canonical `liveDocs` DocSet. This does not need to be volatile. It may be set
+   * multiple times, but should always be set to the same value, as all set values should pass
+   * through `liveDocsCache.computeIfAbsent`
+   */
+  private BitDocSet liveDocs;
+
+  private static final BitDocSet EMPTY = new BitDocSet(new FixedBitSet(0), 0);
+
+  private BitDocSet computeLiveDocs() {
+    switch (leafContexts.size()) {
+      case 0:
+        assert numDocs() == 0;
+        return EMPTY;
+      case 1:
+        final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+        final FixedBitSet fbs;
+        if (onlySegLiveDocs == null) {
+          // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all bits
+          final int onlySegMaxDoc = maxDoc();
+          fbs = new FixedBitSet(onlySegMaxDoc);
+          fbs.set(0, onlySegMaxDoc);
+        } else {
+          fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+        }
+        assert fbs.cardinality() == numDocs();
+        return new BitDocSet(fbs, numDocs());
+      default:
+        final FixedBitSet bs = new FixedBitSet(maxDoc());
+        for (LeafReaderContext ctx : leafContexts) {
+          final LeafReader r = ctx.reader();
+          final Bits segLiveDocs = r.getLiveDocs();
+          final int segDocBase = ctx.docBase;
+          if (segLiveDocs == null) {
+            // `LeafReader.getLiveDocs()` returns null if no deleted docs -- accordingly, set all
+            // bits in seg range
+            bs.set(segDocBase, segDocBase + r.maxDoc());
+          } else {
+            DocSetUtil.copyTo(segLiveDocs, 0, r.maxDoc(), bs, segDocBase);
+          }
+        }
+        assert bs.cardinality() == numDocs();
+        return new BitDocSet(bs, numDocs());
+    }
+  }
+
+  private BitDocSet populateLiveDocs(Supplier<BitDocSet> liveDocsSupplier) {
+    final boolean computeInline;
+    final CompletableFuture<BitDocSet> liveDocsCacheInstance;
+    synchronized (liveDocsCache) {
+      if (liveDocsCache[0] != null) {
+        computeInline = false;
+        liveDocsCacheInstance = liveDocsCache[0];
+      } else {
+        computeInline = true;
+        liveDocsCacheInstance = new CompletableFuture<>();
+        liveDocsCache[0] = liveDocsCacheInstance;
+      }
+    }
+    final BitDocSet docs;
+    if (computeInline) {
+      docs = liveDocsSupplier.get();
+      liveDocsCacheInstance.complete(docs);
+      liveDocs = docs;
+      liveDocsInsertsCount.increment();
+    } else {

Review comment:
       Well I guess the only downside of running the computation within the synchronized block is that you lose the metrics distinction between `liveDocsHitCount` and `liveDocsAsyncHitCount`. That's probably mainly a curiosity now ("during the initialization of liveDocs, approximately how many redundant livedocs computations were avoided?"); the initial hope was to use that metrics distinction in tests to verify that requests which _would_ have resulted in redundant livedocs computations were in fact waiting on a single computation. But I kind of dropped that -- see [here](https://github.com/apache/solr/blob/91222454725122a76399348eb3143e373d2b222c/solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java#L445-L460) and [here](#discussion_r806102113).
   
   If deciding to move computation inside the synchronized block, I think it'd make sense to remove the metrics distinction, since iiuc it would no longer be measuring anything meaningful.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838856353



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -118,10 +122,22 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /** Special-case cache to handle the lazy-init of {@link #liveDocs}. */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private final CompletableFuture<BitDocSet>[] liveDocsCache = new CompletableFuture[1];

Review comment:
       Both SuppressWarnings are necessary with the "internal-one-element-array-for-synchromization" approach, because of the inability to do parameterized array creation. Is there another alternative that I'm missing there?
   
   I'd be ok with Future if you prefer; the only benefit I can see to CompletableFuture on the instance variable (which is private, so arguably is itself an implementation detail, albeit in a very large class :slightly_smiling_face:) is that it provides the `join()` method to get the computation results without forcing us to explicitly handle checked exceptions from the `get()` method.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r800091578



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -865,9 +936,26 @@ public BitDocSet getLiveDocSet() throws IOException {
     // Going through the filter cache will provide thread safety here if we only had getLiveDocs,
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
-    if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+    if (docs != null) {
+      matchAllDocsCacheHitCount.incrementAndGet();
+    } else {
+      synchronized (matchAllDocsCacheComputationTracker) {
+        if (liveDocsFuture != null) {
+          // use future if it already exists
+          assert matchAllDocsCacheComputationTracker.incrementAndGet() > 0;
+        } else {
+          // otherwise create the initial/only future, and run it inline
+          assert matchAllDocsCacheComputationTracker.getAndSet(0) == Long.MIN_VALUE;
+          liveDocsFuture = new FutureTask<>(this::computeLiveDocs);
+          liveDocsFuture.run(); // first caller will block execution here
+        }
+      }
+      try {
+        docs = liveDocsFuture.get(); // subsequent callers block here, waiting for initial/only execution to complete

Review comment:
       resolved this with 779b69b114fd3b7dadb7adf203fd4c7eef5f75ec




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r802960783



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       :+1: 87c9a13871787753947375d23dbb8aa855b17b78
   
   I tried checking here that this wasn't going to magically mess something up downstream (e.g., via some hard check for `instanceof ConstantScoreQuery`). Looks safe, and if you feel confident in this change that works for me. 




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r802986004



##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -54,23 +54,7 @@ public static boolean isNegative(Query q) {
    * varying score (i.e., it is a constant score query).
    */
   public static boolean isConstantScoreQuery(Query q) {
-    return isConstantScoreQuery(q, null);
-  }
-
-  private static Map<Query, Void> lazyInitSeen(Map<Query, Void> seen, Query add) {
-    if (seen == null) {
-      seen = new IdentityHashMap<>();
-    }
-    seen.put(add, null);
-    return seen;
-  }
-
-  /**
-   * Returns true if the specified query is guaranteed to assign the same score to all docs; otherwise false
-   * @param q query to be evaluated
-   * @param seen used to detect possible loops in nested query input
-   */
-  private static boolean isConstantScoreQuery(Query q, Map<Query, Void> seen) {
+    Map<Query, Void> seen = null; // lazy-init; this will be unnecessary in many cases

Review comment:
       Oops, was adding to `seen` but never checking it. The purpose should be clarified by 6074c5dc80a1984de74254b88fa7a748c1624735.
   
   It is possible in theory to create a `WrappedQuery` that at some level wraps itself (perhaps indirectly). I'd be more than happy to get rid of this ... do you think it would be safe to do so?




-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r804321053



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1385,47 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {

Review comment:
       Thanks; I'll look into this after that PR is merged. 




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r801240516



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -2281,6 +2455,11 @@ public void initializeMetrics(SolrMetricsContext parentContext, String scope) {
     parentContext.gauge(() -> openTime, true, "openedAt", Category.SEARCHER.toString(), scope);
     parentContext.gauge(() -> warmupTime, true, "warmupTime", Category.SEARCHER.toString(), scope);
     parentContext.gauge(() -> registerTime, true, "registeredAt", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(fullSortCount::get, true, "fullSortCount", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(skipSortCount::get, true, "skipSortCount", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(matchAllDocsCacheConsultationCount::get, true, "matchAllDocsCacheConsultationCount", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(matchAllDocsCacheHitCount::get, true, "matchAllDocsCacheHitCount", Category.SEARCHER.toString(), scope);
+    parentContext.gauge(matchAllDocsCacheComputationTracker::get, true, "matchAllDocsCacheComputationTracker", Category.SEARCHER.toString(), scope);

Review comment:
       Yes, I went the 1-element cache route and I like it! Way easier and cleaner, I think. I did introduce a couple of additional metrics, including `asyncHits` directly in CaffeineCache, to track the impact of "async" operation (see SOLR-15555). Basically the point of all of these metrics were to "whitebox"-test assumptions about how caching/sort optimizations work. None of them are strictly necessary, but I think they could be useful?




-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r802973324



##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -54,23 +54,7 @@ public static boolean isNegative(Query q) {
    * varying score (i.e., it is a constant score query).
    */
   public static boolean isConstantScoreQuery(Query q) {
-    return isConstantScoreQuery(q, null);
-  }
-
-  private static Map<Query, Void> lazyInitSeen(Map<Query, Void> seen, Query add) {
-    if (seen == null) {
-      seen = new IdentityHashMap<>();
-    }
-    seen.put(add, null);
-    return seen;
-  }
-
-  /**
-   * Returns true if the specified query is guaranteed to assign the same score to all docs; otherwise false
-   * @param q query to be evaluated
-   * @param seen used to detect possible loops in nested query input
-   */
-  private static boolean isConstantScoreQuery(Query q, Map<Query, Void> seen) {
+    Map<Query, Void> seen = null; // lazy-init; this will be unnecessary in many cases

Review comment:
       I'm missing the point of "seen"?  Ideally this method is so simple to not create such tracking data structures.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r806141738



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");

Review comment:
       FWIW I prefer readability in tests (String literals are plainly obvious).  And Yonik once pointed out that avoiding such constants in tests helps ensure we don't change something without seeing the impact (e.g. a client may depend on it).




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r799827730



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+      switch (leafContexts.size()) {
+        case 0:
+          assert numDocs() == 0;
+          docs = new BitDocSet(BitDocSet.empty().getFixedBitSet(), 0);

Review comment:
       This is confusing about why we need to wrap a `empty()` in a new BitDocSet. Clearer if we use the super type.
   
   ```suggestion
             docs = new BitDocSet(DocSet.empty().getFixedBitSet(), 0);
   ```

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+      switch (leafContexts.size()) {
+        case 0:
+          assert numDocs() == 0;
+          docs = new BitDocSet(BitDocSet.empty().getFixedBitSet(), 0);
+          break;
+        case 1:
+          final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+          final FixedBitSet fbs;
+          if (onlySegLiveDocs == null) {
+            final int onlySegMaxDoc = maxDoc();
+            fbs = new FixedBitSet(onlySegMaxDoc);
+            fbs.set(0, onlySegMaxDoc);

Review comment:
       Can you add a comment mentioning the contract of LeafReader.getLiveDocs that if it returns null then we set all the docs live?

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {

Review comment:
       Is there a race condition here when multiple match all docs queries come in at the same time? I think that before if there were multiple queries here at the same time then they would all sync on the same result in the filterCache. Hard to tell, this is tricky logic. Probably good to add a test or two?

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+      switch (leafContexts.size()) {
+        case 0:
+          assert numDocs() == 0;
+          docs = new BitDocSet(BitDocSet.empty().getFixedBitSet(), 0);
+          break;
+        case 1:
+          final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+          final FixedBitSet fbs;
+          if (onlySegLiveDocs == null) {
+            final int onlySegMaxDoc = maxDoc();
+            fbs = new FixedBitSet(onlySegMaxDoc);
+            fbs.set(0, onlySegMaxDoc);
+          } else {
+            fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+          }
+          assert fbs.cardinality() == numDocs();
+          docs = new BitDocSet(fbs, numDocs());
+          break;
+        default:
+          final FixedBitSet bs = new FixedBitSet(maxDoc());
+          for (LeafReaderContext ctx : leafContexts) {
+            final LeafReader r = ctx.reader();
+            final Bits segLiveDocs = r.getLiveDocs();
+            final int segDocBase = ctx.docBase;
+            int segOrd = r.maxDoc() - 1;
+            if (segLiveDocs == null) {
+              do {
+                bs.set(segDocBase + segOrd);

Review comment:
       Can we call the two-argument version of bs.set here? It looks like the Lucene version does some magic to be a bit faster than a straight loop.

##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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 java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {

Review comment:
       👍 

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -866,8 +876,48 @@ public BitDocSet getLiveDocSet() throws IOException {
     // but the addition of setLiveDocs means we needed to add volatile to "liveDocs".
     BitDocSet docs = liveDocs;
     if (docs == null) {
-      //note: maybe should instead calc manually by segment, using FixedBitSet.copyOf(segLiveDocs); avoid filter cache?
-      liveDocs = docs = getDocSetBits(matchAllDocsQuery);
+      switch (leafContexts.size()) {
+        case 0:
+          assert numDocs() == 0;
+          docs = new BitDocSet(BitDocSet.empty().getFixedBitSet(), 0);
+          break;
+        case 1:
+          final Bits onlySegLiveDocs = leafContexts.get(0).reader().getLiveDocs();
+          final FixedBitSet fbs;
+          if (onlySegLiveDocs == null) {
+            final int onlySegMaxDoc = maxDoc();
+            fbs = new FixedBitSet(onlySegMaxDoc);
+            fbs.set(0, onlySegMaxDoc);
+          } else {
+            fbs = FixedBitSet.copyOf(onlySegLiveDocs);
+          }
+          assert fbs.cardinality() == numDocs();
+          docs = new BitDocSet(fbs, numDocs());
+          break;
+        default:
+          final FixedBitSet bs = new FixedBitSet(maxDoc());
+          for (LeafReaderContext ctx : leafContexts) {
+            final LeafReader r = ctx.reader();
+            final Bits segLiveDocs = r.getLiveDocs();
+            final int segDocBase = ctx.docBase;
+            int segOrd = r.maxDoc() - 1;
+            if (segLiveDocs == null) {
+              do {
+                bs.set(segDocBase + segOrd);
+              } while (segOrd-- > 0);
+            } else {
+              do {
+                if (segLiveDocs.get(segOrd)) {
+                  bs.set(segDocBase + segOrd);

Review comment:
       Same here, two argument version.

##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1299,6 +1302,41 @@ public DocList getDocList(Query query, List<Query> filterList, Sort lsort, int o
   public static final int GET_DOCLIST = 0x02; // get the documents actually returned in a response
   public static final int GET_SCORES = 0x01;
 
+  private static boolean isConstantScoreQuery(Query q) {
+    if (q instanceof BoostQuery) {
+      // ConstantScoreQueries are often (always?) wrapped in BoostQuery to assign specific score

Review comment:
       Ok, thanks for checking. We might also want to check for `WrappedQuery`? Not sure.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r806175843



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors
+      }
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); // tasks should already have completed
+    }
+    final SolrCore core = h.getCore();
+    Map<String, Object> liveDocsCacheMetrics = coreToLiveDocsCacheMetrics(core);
+    long inserts = (long) liveDocsCacheMetrics.get("inserts"); // the one and only liveDocs computation
+    long hits = (long) liveDocsCacheMetrics.get("hits"); // hits during the initial phase
+    long asyncHits = (long) liveDocsCacheMetrics.get("asyncHits");

Review comment:
       Yea, I can go either way. My thought was that if the constant ever moves, or the metric path moves, we'd want tests to fail compilation until they are updated. But this makes sense too. I'm not committed to either way.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on pull request #592:
URL: https://github.com/apache/solr/pull/592#issuecomment-1034056219


   Dropped recursing into BooleanQuery in 9c62f9f5f919ecd1c263f13b16c9e89edd1a84a1.


-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r803012235



##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -54,23 +54,7 @@ public static boolean isNegative(Query q) {
    * varying score (i.e., it is a constant score query).
    */
   public static boolean isConstantScoreQuery(Query q) {
-    return isConstantScoreQuery(q, null);
-  }
-
-  private static Map<Query, Void> lazyInitSeen(Map<Query, Void> seen, Query add) {
-    if (seen == null) {
-      seen = new IdentityHashMap<>();
-    }
-    seen.put(add, null);
-    return seen;
-  }
-
-  /**
-   * Returns true if the specified query is guaranteed to assign the same score to all docs; otherwise false
-   * @param q query to be evaluated
-   * @param seen used to detect possible loops in nested query input
-   */
-  private static boolean isConstantScoreQuery(Query q, Map<Query, Void> seen) {
+    Map<Query, Void> seen = null; // lazy-init; this will be unnecessary in many cases

Review comment:
       >Yes definitely; it's simply wrong to create a circular reference in the first place. It's very unreasonable to expect every consumer to have to check for this.
   
   perfect, thanks; this makes sense




-- 
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@solr.apache.org

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



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


[GitHub] [solr] dsmiley commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r802988219



##########
File path: solr/core/src/java/org/apache/solr/search/QueryUtils.java
##########
@@ -54,23 +54,7 @@ public static boolean isNegative(Query q) {
    * varying score (i.e., it is a constant score query).
    */
   public static boolean isConstantScoreQuery(Query q) {
-    return isConstantScoreQuery(q, null);
-  }
-
-  private static Map<Query, Void> lazyInitSeen(Map<Query, Void> seen, Query add) {
-    if (seen == null) {
-      seen = new IdentityHashMap<>();
-    }
-    seen.put(add, null);
-    return seen;
-  }
-
-  /**
-   * Returns true if the specified query is guaranteed to assign the same score to all docs; otherwise false
-   * @param q query to be evaluated
-   * @param seen used to detect possible loops in nested query input
-   */
-  private static boolean isConstantScoreQuery(Query q, Map<Query, Void> seen) {
+    Map<Query, Void> seen = null; // lazy-init; this will be unnecessary in many cases

Review comment:
       Yes definitely; it's simply wrong to create a circular reference in the first place.  It's very unreasonable to expect every consumer to have to check for this.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808356737



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -119,10 +128,18 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /**
+   * Special-case cache to handle the lazy-init of {@link #liveDocs}.
+   */
+  private final SolrCache<MatchAllDocsQuery,BitDocSet> liveDocsCache;

Review comment:
       Yeah; it started out as something ostensibly "simpler" like that, but once you layer in the metrics (mainly to support tests) it becomes less simple. I wonder how succinct it could be, maybe worth a shot.
   
   I'll mull this; if it's something you feel (somewhat) strongly about I'll take a crack at it.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r808376655



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.
+    final String uffsq = System.getProperty(TEST_UFFSQ_PROPNAME, Boolean.toString(random().nextBoolean()));
+    USE_FILTER_FOR_SORTED_QUERY = Boolean.parseBoolean(uffsq);
+    RESTORE_UFFSQ_PROP = System.setProperty(TEST_UFFSQ_PROPNAME, uffsq);
+    initCore("solrconfig-deeppaging.xml", "schema-sorts.xml");
+    createIndex();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (RESTORE_UFFSQ_PROP == null) {
+      System.clearProperty(TEST_UFFSQ_PROPNAME);
+    } else {
+      System.setProperty(TEST_UFFSQ_PROPNAME, RESTORE_UFFSQ_PROP);
+    }
+  }
+
+  public static void createIndex() {
+    for (int i = 0; i < MOST_DOCS; i++) {
+      assertU(adoc("id", Integer.toString(i), "str", "d" + i));
+      if (random().nextInt(MOST_DOCS) == 0) {
+        assertU(commit());  // sometimes make multiple segments
+      }
+    }
+    // add an extra doc to distinguish scoring query from `*:*`
+    assertU(adoc("id", Integer.toString(MOST_DOCS), "str", "e" + MOST_DOCS));
+    assertU(commit());
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    // testing caching, it's far simpler to just reload the core every time to prevent
+    // subsequent requests from affecting each other
+    h.reload();
+  }
+
+  private static long coreToInserts(SolrCore core) {
+    return (long)((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache")).getGauge())
+            .getValue().get("inserts");
+  }
+
+  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher." + skipOrFull + "SortCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToLiveDocsNaiveCacheHitCount(SolrCore core) {
+    return (long)((SolrMetricManager.GaugeWrapper<?>)core
+            .getCoreMetricManager().getRegistry().getMetrics().get("SEARCHER.searcher.liveDocsNaiveCacheHitCount")).getGauge()
+            .getValue();
+  }
+
+  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
+    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
+  }
+
+  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) {
+    return ((MetricsMap)((SolrMetricManager.GaugeWrapper<?>)core.getCoreMetricManager().getRegistry()
+            .getMetrics().get("CACHE.searcher.liveDocsCache")).getGauge()).getValue();
+  }
+  private static final String SCORING_QUERY = "str:d*";
+  private static final String CONSTANT_SCORE_QUERY = "(" + SCORING_QUERY + ")^=1.0"; // wrapped as a ConstantScoreQuery
+  private static final String MATCH_ALL_DOCS_QUERY = "*:*";
+
+  private static final String[] ALL_QUERIES = new String[] { SCORING_QUERY, CONSTANT_SCORE_QUERY, MATCH_ALL_DOCS_QUERY };
+
+  @Test
+  public void testScoringQuery() throws Exception {
+    // plain request should have no caching or sorting optimization
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true"));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testConstantScoreFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable caching and sorting optimizations
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    assertMetricCounts(response, false, 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryNonScoreSort() throws Exception {
+    // plain request with no score in sort should consult filterCache, but need full sorting
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  @Test
+  public void testScoringQueryZeroRows() throws Exception {
+    // always hit cache, optimize sort because rows=0
+    String response = JQ(req("q", SCORING_QUERY, "indent", "true", "rows", "0", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreSortByScore() throws Exception {
+    // hit cache and skip sort because constant score query
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true"));
+    final int insertAndSkipCount = USE_FILTER_FOR_SORTED_QUERY ? 1 : 0;
+    assertMetricCounts(response, false, insertAndSkipCount, USE_FILTER_FOR_SORTED_QUERY ? 0 : 1, insertAndSkipCount);
+  }
+
+  @Test
+  public void testConstantScoreNonScoreSort() throws Exception {
+    // consult filterCache because constant score query, but no skip sort (because sort-by-id)
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, false, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0);
+  }
+
+  /**
+   * As {@link #testConstantScoreNonScoreSort} (though an analogous test could be written corresponding to
+   * {@link #testConstantScoreSortByScore()}, etc...); but with an additional constant-score clause that causes
+   * the associated DocSet, (if {@link #USE_FILTER_FOR_SORTED_QUERY}==true) to be cached as equivalent to
+   * MatchAllDocsQuery/liveDocs, _in addition to_ in the filterCache.
+   *
+   * This is an edge case, but it's the behavior we want, and despite there being two entries, the actual DocSet
+   * will be the same (`==`) in both locations (liveDocs and filterCache)
+   */
+  @Test
+  public void testConstantScoreMatchesAllDocsNonScoreSort() throws Exception {
+    String response = JQ(req("q", CONSTANT_SCORE_QUERY + " OR (str:e*)^=4.0", "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, USE_FILTER_FOR_SORTED_QUERY, USE_FILTER_FOR_SORTED_QUERY ? 1 : 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsPlain() throws Exception {
+    // plain request with "score" sort should skip sort even if `rows` requested
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsFlScore() throws Exception {
+    // explicitly requesting scores should unconditionally disable all cache consultation and sort optimization
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "fl", "id,score", "sort", (random().nextBoolean() ? "id asc" : "score desc")));
+    // NOTE: pretend we're not MatchAllDocs ...
+    assertMetricCounts(response, false, 0, 1, 0, ALL_DOCS);
+  }
+
+  @Test
+  public void testMatchAllDocsZeroRows() throws Exception {
+    // plain request should _always_ skip sort when `rows=0`
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "rows", "0", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 0, 1);
+  }
+
+  @Test
+  public void testMatchAllDocsNonScoreSort() throws Exception {
+    // plain request _with_ rows and non-score sort should consult cache, but not skip sort
+    String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "indent", "true", "sort", "id asc"));
+    assertMetricCounts(response, true, 0, 1, 0);
+  }
+
+  @Test
+  public void testCursorMark() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    boolean includeScoreInSort = random().nextBoolean();
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "sort", includeScoreInSort ? "score desc,id asc" : "id asc"));
+    final int expectNumFound = MATCH_ALL_DOCS_QUERY.equals(q) ? ALL_DOCS : MOST_DOCS;
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    if (includeScoreInSort) {
+      consultMatchAllDocs = false;
+      insertFilterCache = false;
+    } else if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, 1, 0, expectNumFound);
+  }
+
+  @Test
+  public void testCursorMarkZeroRows() throws Exception {
+    String q = pickRandom(ALL_QUERIES);
+    String response = JQ(req("q", q, "indent", "true", "cursorMark", "*", "rows", "0", "sort", random().nextBoolean() ? "id asc" : "score desc,id asc"));
+    final boolean consultMatchAllDocs;
+    final boolean insertFilterCache;
+    final boolean skipSort;
+    if (MATCH_ALL_DOCS_QUERY.equals(q)) {
+      consultMatchAllDocs = true;
+      insertFilterCache = false;
+      skipSort = true;
+    } else {
+      consultMatchAllDocs = false;
+      insertFilterCache = USE_FILTER_FOR_SORTED_QUERY;
+      skipSort = USE_FILTER_FOR_SORTED_QUERY;
+    }
+    assertMetricCounts(response, consultMatchAllDocs, insertFilterCache ? 1 : 0, skipSort ? 0 : 1, skipSort ? 1 : 0);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount, int expectFullSortCount, int expectSkipSortCount) {
+    assertMetricCounts(response, matchAllDocs, expectFilterCacheInsertCount, expectFullSortCount, expectSkipSortCount, matchAllDocs ? ALL_DOCS : MOST_DOCS);
+  }
+
+  private static void assertMetricCounts(String response, boolean matchAllDocs, int expectFilterCacheInsertCount,
+                                         int expectFullSortCount, int expectSkipSortCount, int expectNumFound) {
+    Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+    Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+    SolrCore core = h.getCore();
+    assertEquals("Bad matchAllDocs insert count", (matchAllDocs ? 1 : 0), coreToMatchAllDocsInsertCount(core));
+    assertEquals("Bad filterCache insert count", expectFilterCacheInsertCount, coreToInserts(core));
+    assertEquals("Bad full sort count", expectFullSortCount, coreToSortCount(core, "full"));
+    assertEquals("Bad skip sort count", expectSkipSortCount, coreToSortCount(core, "skip"));
+    assertEquals("Should have exactly " + expectNumFound, expectNumFound, (long) (body.get("numFound"))); // sanity check
+  }
+
+  @Test
+  public void testConcurrentMatchAllDocsInitialization() throws Exception {
+    final int nThreads = 20;
+    final ExecutorService executor = ExecutorUtil.newMDCAwareFixedThreadPool(nThreads, new SolrNamedThreadFactory(getTestName()));
+    final Future<?>[] followup = new Future<?>[nThreads];
+    for (int i = 0; i < nThreads; i++) {
+      final int myI = i;
+      followup[i] = executor.submit(() -> {
+        try {
+          // NOTE: we use cursorMark=* here because it prevents consulting the queryResultCache, which can interfere
+          // with DocSet fetching (which is what we care about in this test).
+          String response = JQ(req("q", MATCH_ALL_DOCS_QUERY, "request_id", Integer.toString(myI), "cursorMark", "*", "sort", "id asc"));
+          Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
+          Map<?, ?> body = (Map<?, ?>) (res.get("response"));
+          assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) (body.get("numFound"))); // sanity check
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+      });
+    }
+    try {
+      for (Future<?> f : followup) {
+        f.get(); // to access exceptions/errors

Review comment:
       I'm curious why? If the issue is _swallowing_ errors/exceptions, these would not be returned by `.get()` anyway, but will instead be thrown wrapped in `ExecutionException`, iiuc.




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r812356681



##########
File path: solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
##########
@@ -0,0 +1,313 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.fromJSONString;
+
+/**
+ * Verify caching interactions between main query and filterCache
+ */
+public class TestMainQueryCaching extends SolrTestCaseJ4 {
+
+  private static final int MOST_DOCS = 100;
+  private static final int ALL_DOCS = MOST_DOCS + 1;
+  private static final String TEST_UFFSQ_PROPNAME = "solr.test.useFilterForSortedQuery";
+  static String RESTORE_UFFSQ_PROP;
+  static boolean USE_FILTER_FOR_SORTED_QUERY;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // TODO: figure out why the line below (accepting this property as overridden on test invocation) isn't working
+    //  as expected.

Review comment:
       Removed this with d87801882fae79b391080cee735dfc615e585fbf; it's really orthogonal to this PR




-- 
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@solr.apache.org

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



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


[GitHub] [solr] madrob commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r798975829



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -1384,16 +1385,24 @@ private void getDocListC(QueryResult qr, QueryCommand cmd) throws IOException {
     // - we don't want score returned.
 
     // check if we should try and use the filter cache
-    boolean useFilterCache = false;
-    if ((flags & (GET_SCORES | NO_CHECK_FILTERCACHE)) == 0 && useFilterForSortedQuery && cmd.getSort() != null
-        && filterCache != null) {
-      useFilterCache = true;
-      SortField[] sfields = cmd.getSort().getSort();
-      for (SortField sf : sfields) {
-        if (sf.getType() == SortField.Type.SCORE) {
-          useFilterCache = false;
-          break;
-        }
+    final boolean needSort;
+    final boolean useFilterCache;
+    if ((flags & (GET_SCORES | NO_CHECK_FILTERCACHE)) != 0 || filterCache == null) {
+      needSort = true; // this value should be irrelevant when `useFilterCache=false`
+      useFilterCache = false;
+    } else {
+      final Sort sort;
+      if (q instanceof  MatchAllDocsQuery // special-case MatchAllDocsQuery: implicit default useFilterForSortedQuery=true
+              || (useFilterForSortedQuery && q instanceof ConstantScoreQuery)) { // default behavior should not risk filterCache thrashing
+        final SortField[] sortFields;
+        // We only need to sort if we're returning results AND sorting by something other than SCORE (sort by
+        // "score" alone is pointless for these constant score queries)
+        needSort = cmd.getLen() > 0 && (sort = cmd.getSort()) != null && ((sortFields = sort.getSort()).length > 1 || sortFields[0].getType() != Type.SCORE);

Review comment:
       This complex conditional with nested assignments is fairly difficult to read, can we expand to something less terse?




-- 
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@solr.apache.org

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



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


[GitHub] [solr] magibney commented on a change in pull request #592: SOLR-14765: Optimize DocList creation by skipping sort for sort-irrelevant cases

Posted by GitBox <gi...@apache.org>.
magibney commented on a change in pull request #592:
URL: https://github.com/apache/solr/pull/592#discussion_r838873317



##########
File path: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
##########
@@ -118,10 +122,22 @@
   private final int queryResultMaxDocsCached;
   private final boolean useFilterForSortedQuery;
 
+  /** Special-case cache to handle the lazy-init of {@link #liveDocs}. */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private final CompletableFuture<BitDocSet>[] liveDocsCache = new CompletableFuture[1];

Review comment:
       >Both SuppressWarnings are necessary
   
   to be more specific: gradle won't compile the project unless I add these two SuppressWarnings.




-- 
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@solr.apache.org

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



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