You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ab...@apache.org on 2024/03/01 10:22:35 UTC

(solr) branch branch_9x updated: SOLR-17172: Add QueryLimits termination to the existing heavy SearchComponent-s (#2290) (#2323)

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

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


The following commit(s) were added to refs/heads/branch_9x by this push:
     new e7829d42edb SOLR-17172: Add QueryLimits termination to the existing heavy SearchComponent-s (#2290) (#2323)
e7829d42edb is described below

commit e7829d42edb436afa32151da7bcabc87259385a5
Author: Andrzej BiaƂecki <ab...@apache.org>
AuthorDate: Fri Mar 1 11:22:29 2024 +0100

    SOLR-17172: Add QueryLimits termination to the existing heavy SearchComponent-s (#2290) (#2323)
---
 solr/CHANGES.txt                                   |   3 +
 .../apache/solr/handler/MoreLikeThisHandler.java   |   3 +
 .../apache/solr/handler/RequestHandlerBase.java    |  10 +-
 .../solr/handler/component/ExpandComponent.java    |   8 ++
 .../solr/handler/component/FacetComponent.java     |  13 ++-
 .../solr/handler/component/HighlightComponent.java |   3 +
 .../handler/component/MoreLikeThisComponent.java   |  10 ++
 .../solr/handler/component/QueryComponent.java     |   7 +-
 .../solr/handler/component/ResponseBuilder.java    |   4 +-
 .../solr/handler/component/SearchHandler.java      |   9 +-
 .../handler/component/SpellCheckComponent.java     |  21 ++++-
 .../solr/handler/component/SuggestComponent.java   |  30 ++++--
 .../org/apache/solr/request/SolrRequestInfo.java   |   5 +-
 .../apache/solr/response/SolrQueryResponse.java    |  43 ++++++++-
 .../java/org/apache/solr/search/QueryLimits.java   | 104 ++++++++++++++++----
 .../solr/search/QueryLimitsExceededException.java  |  26 +++++
 .../org/apache/solr/search/SolrIndexSearcher.java  |  21 ++---
 .../org/apache/solr/search/facet/FacetModule.java  |   5 +-
 .../SearchGroupShardResponseProcessor.java         |   6 +-
 .../TopGroupsShardResponseProcessor.java           |   6 +-
 .../apache/solr/spelling/SpellCheckCollator.java   |   6 ++
 .../java/org/apache/solr/util/TestInjection.java   |   4 +
 .../test/org/apache/solr/TestGroupingSearch.java   |  22 ++++-
 .../solr/search/CallerSpecificQueryLimit.java      | 105 +++++++++++++++++++++
 .../org/apache/solr/search/TestQueryLimits.java    |  92 ++++++++++++++++++
 .../handler/clustering/ClusteringComponent.java    |  10 +-
 .../query-guide/pages/common-query-parameters.adoc |  24 ++---
 .../org/apache/solr/client/solrj/SolrQuery.java    |  23 +++++
 .../apache/solr/common/params/CommonParams.java    |   6 ++
 29 files changed, 535 insertions(+), 94 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5792622c3d1..71be998126c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -33,6 +33,9 @@ Improvements
 
 * SOLR-16138: Throw a exception when issuing a streaming expression and all cores are down instead of returning 0 documents. (Antoine Bursaux via Eric Pugh)
 
+* SOLR-17172: Add QueryLimits termination to the existing heavy SearchComponent-s. This allows query limits (e.g. timeAllowed,
+  cpuAllowed) to terminate expensive operations within components if limits are exceeded. (Andrzej Bialecki)
+
 Optimizations
 ---------------------
 * SOLR-17144: Close searcherExecutor thread per core after 1 minute (Pierre Salagnac, Christine Poerschke)
diff --git a/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java b/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
index 4cb201b0287..2027012d23a 100644
--- a/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
@@ -61,6 +61,7 @@ import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocListAndSet;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.QParserPlugin;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.QueryParsing;
 import org.apache.solr.search.QueryUtils;
 import org.apache.solr.search.ReturnFields;
@@ -265,6 +266,8 @@ public class MoreLikeThisHandler extends RequestHandlerBase {
       }
     } catch (ExitableDirectoryReader.ExitingReaderException ex) {
       log.warn("Query: {}; ", req.getParamString(), ex);
+      QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+      queryLimits.maybeExitWithPartialResults("MoreLikeThis");
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index e700428f1a4..d6a7334449a 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -225,13 +225,9 @@ public abstract class RequestHandlerBase
       rsp.setHttpCaching(httpCaching);
       handleRequestBody(req, rsp);
       // count timeouts
-      NamedList<?> header = rsp.getResponseHeader();
-      if (header != null) {
-        if (Boolean.TRUE.equals(
-            header.getBooleanArg(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY))) {
-          metrics.numTimeouts.mark();
-          rsp.setHttpCaching(false);
-        }
+      if (rsp.isPartialResults()) {
+        metrics.numTimeouts.mark();
+        rsp.setHttpCaching(false);
       }
     } catch (Exception e) {
       e = normalizeReceivedException(req, e);
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
index 560c08b5dad..761f59df228 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
@@ -79,6 +79,7 @@ import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocSlice;
 import org.apache.solr.search.QParser;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.QueryUtils;
 import org.apache.solr.search.ReturnFields;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -253,6 +254,10 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
       // When no context is available we can skip the expanding
       return;
     }
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+    if (queryLimits.maybeExitWithPartialResults("Expand process")) {
+      return;
+    }
 
     boolean nullGroupOnCurrentPage = false;
     int currentContext = 0;
@@ -441,6 +446,9 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
     }
 
     searcher.search(QueryUtils.combineQueryAndFilter(query, pfilter.filter), collector);
+    if (queryLimits.maybeExitWithPartialResults("Expand expand")) {
+      return;
+    }
 
     rb.rsp.add("expanded", groupExpandCollector.getGroups(searcher, rb.rsp.getReturnFields()));
   }
diff --git a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
index 204d221e61a..bf3bb17f833 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
@@ -48,6 +48,7 @@ import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.PointField;
 import org.apache.solr.search.DocSet;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.QueryParsing;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.search.facet.FacetDebugInfo;
@@ -262,6 +263,7 @@ public class FacetComponent extends SearchComponent {
   public void process(ResponseBuilder rb) throws IOException {
 
     if (rb.doFacets) {
+      QueryLimits queryLimits = QueryLimits.getCurrentLimits();
       SolrParams params = rb.req.getParams();
       SimpleFacets f = newSimpleFacets(rb.req, rb.getResults().docSet, params, rb);
 
@@ -275,6 +277,10 @@ public class FacetComponent extends SearchComponent {
       }
 
       NamedList<Object> counts = FacetComponent.getFacetCounts(f, fdebug);
+      rb.rsp.add(FACET_COUNTS_KEY, counts);
+      if (queryLimits.maybeExitWithPartialResults("Faceting counts")) {
+        return;
+      }
       String[] pivots = params.getParams(FacetParams.FACET_PIVOT);
       if (pivots != null && Array.getLength(pivots) != 0) {
         PivotFacetProcessor pivotProcessor =
@@ -283,14 +289,15 @@ public class FacetComponent extends SearchComponent {
         if (v != null) {
           counts.add(PIVOT_KEY, v);
         }
+        if (queryLimits.maybeExitWithPartialResults("Faceting pivots")) {
+          return;
+        }
       }
 
       if (fdebug != null) {
         long timeElapsed = (long) timer.getTime();
         fdebug.setElapse(timeElapsed);
       }
-
-      rb.rsp.add(FACET_COUNTS_KEY, counts);
     }
   }
 
@@ -1171,6 +1178,8 @@ public class FacetComponent extends SearchComponent {
     rb.rsp.add(FACET_COUNTS_KEY, facet_counts);
 
     rb._facetInfo = null; // could be big, so release asap
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+    queryLimits.maybeExitWithPartialResults("Faceting finish");
   }
 
   private SimpleOrderedMap<List<NamedList<Object>>> createPivotFacetOutput(ResponseBuilder rb) {
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java b/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
index 220ff1aca74..d05c44ef539 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
@@ -38,6 +38,7 @@ import org.apache.solr.highlight.UnifiedSolrHighlighter;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.QParserPlugin;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.QueryParsing;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.SolrPluginUtils;
@@ -164,6 +165,8 @@ public class HighlightComponent extends SearchComponent
           // TODO ???? add this directly to the response?
           rb.rsp.add(highlightingResponseField(), convertHighlights(sumData));
         }
+        QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+        queryLimits.maybeExitWithPartialResults("Highlighting process");
       }
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java b/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
index e272c646116..bc1267bfd54 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
@@ -47,6 +47,7 @@ import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocListAndSet;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.ReturnFields;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SolrReturnFields;
@@ -101,6 +102,7 @@ public class MoreLikeThisComponent extends SearchComponent {
           MoreLikeThisHandler.MoreLikeThisHelper mlt =
               new MoreLikeThisHandler.MoreLikeThisHelper(params, searcher);
           NamedList<NamedList<?>> mltQueryByDocKey = new NamedList<>();
+          QueryLimits queryLimits = QueryLimits.getCurrentLimits();
           for (DocIterator results = rb.getResults().docList.iterator(); results.hasNext(); ) {
             int docId = results.nextDoc();
             final List<MoreLikeThisHandler.InterestingTerm> interestingTerms =
@@ -108,6 +110,9 @@ public class MoreLikeThisComponent extends SearchComponent {
             if (interestingTerms.isEmpty()) {
               continue;
             }
+            if (queryLimits.maybeExitWithPartialResults("MoreLikeThis process")) {
+              break;
+            }
             final String uniqueKey = rb.req.getSchema().getUniqueKeyField().getName();
             final Document document = rb.req.getSearcher().doc(docId);
             final String uniqueVal = rb.req.getSchema().printableUniqueKey(document);
@@ -416,6 +421,8 @@ public class MoreLikeThisComponent extends SearchComponent {
       interestingTermsResponse = new SimpleOrderedMap<>();
     }
 
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+
     while (iterator.hasNext()) {
       int id = iterator.nextDoc();
       int rows = p.getInt(MoreLikeThisParams.DOC_COUNT, 5);
@@ -458,6 +465,9 @@ public class MoreLikeThisComponent extends SearchComponent {
           interestingTermsResponse.add(name, interestingTermsString);
         }
       }
+      if (queryLimits.maybeExitWithPartialResults("MoreLikeThis moreLikeThese")) {
+        break;
+      }
     }
     // add debug information
     if (dbg != null) {
diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
index e154b79b1f6..3cde6db4adf 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
@@ -1361,10 +1361,9 @@ public class QueryComponent extends SearchComponent {
           if (Boolean.TRUE.equals(
               responseHeader.getBooleanArg(
                   SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY))) {
-            rb.rsp
-                .getResponseHeader()
-                .asShallowMap()
-                .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+            rb.rsp.setPartialResults();
+            rb.rsp.addPartialResponseDetail(
+                responseHeader.get(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_DETAILS_KEY));
           }
         }
         SolrDocumentList docs =
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java b/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java
index f3e79675414..94331d2ef15 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ResponseBuilder.java
@@ -448,9 +448,7 @@ public class ResponseBuilder {
   public void setResult(QueryResult result) {
     setResults(result.getDocListAndSet());
     if (result.isPartialResults()) {
-      rsp.getResponseHeader()
-          .asShallowMap()
-          .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+      rsp.setPartialResults();
       if (getResults() != null && getResults().docList == null) {
         getResults().docList =
             new DocSlice(0, 0, new int[] {}, new float[] {}, 0, 0, TotalHits.Relation.EQUAL_TO);
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
index a3d0759487e..06bf45d48da 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
@@ -496,10 +496,7 @@ public class SearchHandler extends RequestHandlerBase
           debug.add("explain", new NamedList<>());
           rb.rsp.add("debug", debug);
         }
-        rb.rsp
-            .getResponseHeader()
-            .asShallowMap()
-            .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+        rb.rsp.setPartialResults();
       }
     } else {
       // a distributed request
@@ -591,9 +588,7 @@ public class SearchHandler extends RequestHandlerBase
                 if (allShardsFailed) {
                   throwSolrException(srsp.getException());
                 } else {
-                  rsp.getResponseHeader()
-                      .asShallowMap()
-                      .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+                  rsp.setPartialResults();
                 }
               }
             }
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java b/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
index 14afdb6703f..9a45aefea02 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
@@ -38,7 +38,8 @@ import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.ExitableDirectoryReader;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.spell.SuggestMode;
 import org.apache.lucene.search.spell.SuggestWord;
@@ -58,6 +59,7 @@ import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.DocSet;
 import org.apache.solr.search.QParser;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.spelling.AbstractLuceneSpellChecker;
@@ -116,13 +118,18 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
     if (!params.getBool(COMPONENT_NAME, false)) {
       return;
     }
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
     SolrSpellChecker spellChecker = getSpellChecker(params);
     if (params.getBool(SPELLCHECK_BUILD, false)) {
       spellChecker.build(rb.req.getCore(), rb.req.getSearcher());
       rb.rsp.add("command", "build");
+      queryLimits.maybeExitWithPartialResults(
+          "SpellCheck build " + spellChecker.getDictionaryName());
     } else if (params.getBool(SPELLCHECK_RELOAD, false)) {
       spellChecker.reload(rb.req.getCore(), rb.req.getSearcher());
       rb.rsp.add("command", "reload");
+      queryLimits.maybeExitWithPartialResults(
+          "SpellCheck reload " + spellChecker.getDictionaryName());
     }
   }
 
@@ -174,6 +181,8 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
           hits = hitsLong.longValue();
         }
 
+        QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+
         SpellingResult spellingResult = null;
         if (maxResultsForSuggest == null || hits <= maxResultsForSuggest) {
           SuggestMode suggestMode = SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX;
@@ -183,7 +192,10 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
             suggestMode = SuggestMode.SUGGEST_ALWAYS;
           }
 
-          IndexReader reader = rb.req.getSearcher().getIndexReader();
+          DirectoryReader reader = rb.req.getSearcher().getIndexReader();
+          if (queryLimits.isLimitsEnabled()) {
+            reader = ExitableDirectoryReader.wrap(reader, queryLimits);
+          }
           SpellingOptions options =
               new SpellingOptions(
                   tokens,
@@ -198,6 +210,11 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
         } else {
           spellingResult = new SpellingResult();
         }
+
+        if (queryLimits.maybeExitWithPartialResults("SpellCheck getSuggestions")) {
+          return;
+        }
+
         boolean isCorrectlySpelled =
             hits > (maxResultsForSuggest == null ? 0 : maxResultsForSuggest);
 
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java b/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
index 49bee061259..56b72b75ae6 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
@@ -50,6 +50,7 @@ import org.apache.solr.core.SolrEventListener;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.metrics.SolrMetricsContext;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.spelling.suggest.SolrSuggester;
 import org.apache.solr.spelling.suggest.SuggesterOptions;
@@ -190,17 +191,23 @@ public class SuggestComponent extends SearchComponent
     } else {
       querysuggesters = getSuggesters(params);
     }
-
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
     if (params.getBool(SUGGEST_BUILD, false) || buildAll) {
+      rb.rsp.add("command", (!buildAll) ? "build" : "buildAll");
       for (SolrSuggester suggester : querysuggesters) {
         suggester.build(rb.req.getCore(), rb.req.getSearcher());
+        if (queryLimits.maybeExitWithPartialResults("Suggester build " + suggester.getName())) {
+          return;
+        }
       }
-      rb.rsp.add("command", (!buildAll) ? "build" : "buildAll");
     } else if (params.getBool(SUGGEST_RELOAD, false) || reloadAll) {
+      rb.rsp.add("command", (!reloadAll) ? "reload" : "reloadAll");
       for (SolrSuggester suggester : querysuggesters) {
         suggester.reload();
+        if (queryLimits.maybeExitWithPartialResults("Suggester reload " + suggester.getName())) {
+          return;
+        }
       }
-      rb.rsp.add("command", (!reloadAll) ? "reload" : "reloadAll");
     }
   }
 
@@ -229,7 +236,7 @@ public class SuggestComponent extends SearchComponent
   @Override
   public void process(ResponseBuilder rb) throws IOException {
     SolrParams params = rb.req.getParams();
-    log.info("SuggestComponent process with : {}", params);
+    log.debug("SuggestComponent process with : {}", params);
     if (!params.getBool(COMPONENT_NAME, false) || suggesters.isEmpty()) {
       return;
     }
@@ -272,11 +279,15 @@ public class SuggestComponent extends SearchComponent
               new CharsRef(query), count, contextFilter, allTermsRequired, highlight);
       SimpleOrderedMap<SimpleOrderedMap<NamedList<Object>>> namedListResults =
           new SimpleOrderedMap<>();
+      rb.rsp.add(SuggesterResultLabels.SUGGEST, namedListResults);
+      QueryLimits queryLimits = QueryLimits.getCurrentLimits();
       for (SolrSuggester suggester : querySuggesters) {
         SuggesterResult suggesterResult = suggester.getSuggestions(options);
         toNamedList(suggesterResult, namedListResults);
+        if (queryLimits.maybeExitWithPartialResults("Suggester process " + suggester.getName())) {
+          return;
+        }
       }
-      rb.rsp.add(SuggesterResultLabels.SUGGEST, namedListResults);
     }
   }
 
@@ -290,6 +301,7 @@ public class SuggestComponent extends SearchComponent
     int count = params.getInt(SUGGEST_COUNT, 1);
 
     List<SuggesterResult> suggesterResults = new ArrayList<>();
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
 
     // Collect Shard responses
     for (ShardRequest sreq : rb.finished) {
@@ -306,9 +318,12 @@ public class SuggestComponent extends SearchComponent
           log.info("{} : {}", srsp.getShard(), namedList);
         }
         suggesterResults.add(toSuggesterResult(namedList));
+        // may have tripped the mem limits
+        if (queryLimits.maybeExitWithPartialResults("Suggester finish")) {
+          break;
+        }
       }
     }
-
     // Merge Shard responses
     SuggesterResult suggesterResult = merge(suggesterResults, count);
     SimpleOrderedMap<SimpleOrderedMap<NamedList<Object>>> namedListResults =
@@ -316,6 +331,9 @@ public class SuggestComponent extends SearchComponent
     toNamedList(suggesterResult, namedListResults);
 
     rb.rsp.add(SuggesterResultLabels.SUGGEST, namedListResults);
+
+    // either throw or mark
+    queryLimits.maybeExitWithPartialResults("Suggester finish");
   }
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
index 72194da9641..b7d0a11eb97 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
@@ -245,9 +245,10 @@ public class SolrRequestInfo {
   public QueryLimits getLimits() {
     // make sure the ThreadCpuTime is always initialized
     getThreadCpuTimer();
-    return req == null
+    return req == null || rsp == null
         ? QueryLimits.NONE
-        : (QueryLimits) req.getContext().computeIfAbsent(LIMITS_KEY, (k) -> new QueryLimits(req));
+        : (QueryLimits)
+            req.getContext().computeIfAbsent(LIMITS_KEY, (k) -> new QueryLimits(req, rsp));
   }
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java b/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java
index 55b62eaff37..2f952649fd0 100644
--- a/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java
+++ b/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java
@@ -22,6 +22,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import javax.servlet.http.HttpServletResponse;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.search.ReturnFields;
@@ -60,7 +61,8 @@ import org.apache.solr.search.SolrReturnFields;
  */
 public class SolrQueryResponse {
   public static final String NAME = "response";
-  public static final String RESPONSE_HEADER_PARTIAL_RESULTS_KEY = "partialResults";
+  public static final String RESPONSE_HEADER_PARTIAL_RESULTS_KEY = CommonParams.PARTIAL_RESULTS;
+  public static final String RESPONSE_HEADER_PARTIAL_RESULTS_DETAILS_KEY = "partialResultsDetails";
   public static final String RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY =
       "segmentTerminatedEarly";
   public static final String RESPONSE_HEADER_KEY = "responseHeader";
@@ -135,6 +137,45 @@ public class SolrQueryResponse {
     return returnFields;
   }
 
+  /**
+   * If {@link #getResponseHeader()} is available, set {@link #RESPONSE_HEADER_PARTIAL_RESULTS_KEY}
+   * flag to true.
+   */
+  public void setPartialResults() {
+    NamedList<Object> header = getResponseHeader();
+    if (header != null
+        && header.get(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY) == null) {
+      header.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+    }
+  }
+
+  /**
+   * If {@link #getResponseHeader()} is available, return the value of {@link
+   * #RESPONSE_HEADER_PARTIAL_RESULTS_KEY} or false.
+   */
+  public boolean isPartialResults() {
+    NamedList<Object> header = getResponseHeader();
+    if (header != null) {
+      return Boolean.TRUE.equals(
+          header.getBooleanArg(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY));
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * If {@link #getResponseHeader()} is available, add a reason for returning partial response.
+   *
+   * @param detail reason for returning partial response. Multiple components can add multiple
+   *     reasons at different stages in request processing.
+   */
+  public void addPartialResponseDetail(Object detail) {
+    NamedList<Object> header = getResponseHeader();
+    if (header != null && detail != null) {
+      header.add(RESPONSE_HEADER_PARTIAL_RESULTS_DETAILS_KEY, detail);
+    }
+  }
+
   /**
    * Appends a named value to the list of named values to be returned.
    *
diff --git a/solr/core/src/java/org/apache/solr/search/QueryLimits.java b/solr/core/src/java/org/apache/solr/search/QueryLimits.java
index 83d91621fa3..2e232bccde6 100644
--- a/solr/core/src/java/org/apache/solr/search/QueryLimits.java
+++ b/solr/core/src/java/org/apache/solr/search/QueryLimits.java
@@ -22,7 +22,11 @@ import static org.apache.solr.search.TimeAllowedLimit.hasTimeLimit;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.lucene.index.QueryTimeout;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.TestInjection;
 
 /**
  * Represents the limitations on the query. These limits might be wall clock time, cpu time, memory,
@@ -35,31 +39,89 @@ public class QueryLimits implements QueryTimeout {
 
   public static QueryLimits NONE = new QueryLimits();
 
-  private QueryLimits() {}
+  private final SolrQueryResponse rsp;
+  private final boolean allowPartialResults;
+
+  // short-circuit the checks if any limit has been tripped
+  private volatile boolean limitsTripped = false;
+
+  private QueryLimits() {
+    this(null, null);
+  }
 
   /**
    * Implementors of a Query Limit should add an if block here to activate it, and typically this if
    * statement will hinge on hasXXXLimit() static method attached to the implementation class.
    *
    * @param req the current SolrQueryRequest.
+   * @param rsp the current SolrQueryResponse.
    */
-  public QueryLimits(SolrQueryRequest req) {
-    if (hasTimeLimit(req)) {
-      limits.add(new TimeAllowedLimit(req));
+  public QueryLimits(SolrQueryRequest req, SolrQueryResponse rsp) {
+    this.rsp = rsp;
+    this.allowPartialResults =
+        req != null ? req.getParams().getBool(CommonParams.PARTIAL_RESULTS, true) : true;
+    if (req != null) {
+      if (hasTimeLimit(req)) {
+        limits.add(new TimeAllowedLimit(req));
+      }
+      if (hasCpuLimit(req)) {
+        limits.add(new CpuAllowedLimit(req));
+      }
     }
-    if (hasCpuLimit(req)) {
-      limits.add(new CpuAllowedLimit(req));
+    // for testing
+    if (TestInjection.queryTimeout != null) {
+      limits.add(TestInjection.queryTimeout);
     }
   }
 
   @Override
   public boolean shouldExit() {
+    if (limitsTripped) {
+      return true;
+    }
     for (QueryTimeout limit : limits) {
       if (limit.shouldExit()) {
+        limitsTripped = true;
+        break;
+      }
+    }
+    return limitsTripped;
+  }
+
+  /**
+   * Format an exception message with optional label and details from {@link #limitStatusMessage()}.
+   */
+  public String formatExceptionMessage(String label) {
+    return "Limits exceeded!"
+        + (label != null ? " (" + label + ")" : "")
+        + ": "
+        + limitStatusMessage();
+  }
+
+  /**
+   * If limit is reached then depending on the request param {@link CommonParams#PARTIAL_RESULTS}
+   * either mark it as partial result in the response and signal the caller to return, or throw an
+   * exception.
+   *
+   * @param label optional label to indicate the caller.
+   * @return true if the caller should stop processing and return partial results, false otherwise.
+   * @throws QueryLimitsExceededException if {@link CommonParams#PARTIAL_RESULTS} request parameter
+   *     is false and limits have been reached.
+   */
+  public boolean maybeExitWithPartialResults(String label) throws QueryLimitsExceededException {
+    if (isLimitsEnabled() && shouldExit()) {
+      if (allowPartialResults) {
+        if (rsp != null) {
+          rsp.setPartialResults();
+          rsp.addPartialResponseDetail(formatExceptionMessage(label));
+        }
         return true;
+      } else {
+        throw new QueryLimitsExceededException(formatExceptionMessage(label));
       }
+    } else {
+      return false;
     }
-    return false;
   }
 
   /**
@@ -73,27 +135,31 @@ public class QueryLimits implements QueryTimeout {
    * @return A string describing the state pass/fail state of each limit specified for this request.
    */
   public String limitStatusMessage() {
-    StringBuilder sb = new StringBuilder();
-    boolean first = true;
+    if (limits.isEmpty()) {
+      return "This request is unlimited.";
+    }
+    StringBuilder sb = new StringBuilder("Query limits: ");
     for (QueryTimeout limit : limits) {
-      if (first) {
-        first = false;
-        sb.append("Query limits:");
-      }
       sb.append("[");
       sb.append(limit.getClass().getSimpleName());
       sb.append(":");
       sb.append(limit.shouldExit() ? "LIMIT EXCEEDED" : "within limit");
       sb.append("]");
     }
-    if (sb.length() == 0) {
-      return "This request is unlimited.";
-    } else {
-      return sb.toString();
-    }
+    return sb.toString();
   }
 
-  public boolean isTimeoutEnabled() {
+  /** Return true if there are any limits enabled for the current request. */
+  public boolean isLimitsEnabled() {
     return !limits.isEmpty();
   }
+
+  /**
+   * Helper method to retrieve the current QueryLimits from {@link SolrRequestInfo#getRequestInfo()}
+   * if it exists, otherwise it returns {@link #NONE}.
+   */
+  public static QueryLimits getCurrentLimits() {
+    final SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
+    return info != null ? info.getLimits() : NONE;
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/search/QueryLimitsExceededException.java b/solr/core/src/java/org/apache/solr/search/QueryLimitsExceededException.java
new file mode 100644
index 00000000000..769d06d66e7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/QueryLimitsExceededException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.search;
+
+import org.apache.lucene.index.ExitableDirectoryReader;
+
+public class QueryLimitsExceededException extends ExitableDirectoryReader.ExitingReaderException {
+
+  public QueryLimitsExceededException(String msg) {
+    super(msg);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index a3c7f3c3690..95e66af4aa2 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -204,10 +204,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     assert reader != null;
     reader = UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMapper());
     if (useExitableDirectoryReader) { // SOLR-16693 legacy; may be removed.  Probably inefficient.
-      SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
-      assert requestInfo != null;
-      QueryLimits limits = requestInfo.getLimits();
-      reader = ExitableDirectoryReader.wrap(reader, limits);
+      reader = ExitableDirectoryReader.wrap(reader, QueryLimits.getCurrentLimits());
     }
     return reader;
   }
@@ -717,10 +714,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   @Override
   protected void search(List<LeafReaderContext> leaves, Weight weight, Collector collector)
       throws IOException {
-    SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
-    if (useExitableDirectoryReader
-        || requestInfo == null
-        || !requestInfo.getLimits().isTimeoutEnabled()) {
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+    if (useExitableDirectoryReader || !queryLimits.isLimitsEnabled()) {
       // no timeout.  Pass through to super class
       super.search(leaves, weight, collector);
     } else {
@@ -730,11 +725,11 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       // So we need to make a new IndexSearcher instead of using "this".
       new IndexSearcher(reader) { // cheap, actually!
         void searchWithTimeout() throws IOException {
-          setTimeout(requestInfo.getLimits()); // Lucene's method name is less than ideal here...
+          setTimeout(queryLimits); // Lucene's method name is less than ideal here...
           super.search(leaves, weight, collector); // FYI protected access
           if (timedOut()) {
-            throw new LimitExceededFromScorerException(
-                "Limits exceeded! " + requestInfo.getLimits().limitStatusMessage());
+            throw new QueryLimitsExceededException(
+                "Limits exceeded! (search): " + queryLimits.limitStatusMessage());
           }
         }
       }.searchWithTimeout();
@@ -978,8 +973,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     }
 
     DocSet answer;
-    SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
-    if (requestInfo != null && requestInfo.getLimits().isTimeoutEnabled()) {
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+    if (queryLimits.isLimitsEnabled()) {
       // If there is a possibility of timeout for this query, then don't reserve a computation slot.
       // Further, we can't naively wait for an in progress computation to finish, because if we time
       // out before it does then we won't even have partial results to provide. We could possibly
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
index 78f01106973..5a84339a775 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
@@ -303,10 +303,7 @@ public class FacetModule extends SearchComponent {
         if (Boolean.TRUE.equals(
             shardResponseHeader.getBooleanArg(
                 SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY))) {
-          rb.rsp
-              .getResponseHeader()
-              .asShallowMap()
-              .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+          rb.rsp.setPartialResults();
         }
         continue;
       }
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
index a88817de8ff..35fcf5e4b21 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
@@ -36,7 +36,6 @@ import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.handler.component.ShardRequest;
 import org.apache.solr.handler.component.ShardResponse;
-import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SortSpec;
 import org.apache.solr.search.grouping.distributed.ShardResponseProcessor;
 import org.apache.solr.search.grouping.distributed.command.SearchGroupsFieldCommandResult;
@@ -106,10 +105,7 @@ public class SearchGroupShardResponseProcessor implements ShardResponseProcessor
         shardInfo.add(srsp.getShard(), nl);
       }
       if (ShardParams.getShardsTolerantAsBool(rb.req.getParams()) && srsp.getException() != null) {
-        rb.rsp
-            .getResponseHeader()
-            .asShallowMap()
-            .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+        rb.rsp.setPartialResults();
         continue; // continue if there was an error and we're tolerant.
       }
       maxElapsedTime = Math.max(maxElapsedTime, solrResponse.getElapsedTime());
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java
index af5d19a76e3..0a2eea9b9ee 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java
@@ -38,7 +38,6 @@ import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.handler.component.ShardDoc;
 import org.apache.solr.handler.component.ShardRequest;
 import org.apache.solr.handler.component.ShardResponse;
-import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.Grouping;
 import org.apache.solr.search.SortSpec;
 import org.apache.solr.search.grouping.distributed.ShardResponseProcessor;
@@ -118,10 +117,7 @@ public class TopGroupsShardResponseProcessor implements ShardResponseProcessor {
         shardInfo.add(srsp.getShard(), individualShardInfo);
       }
       if (ShardParams.getShardsTolerantAsBool(rb.req.getParams()) && srsp.getException() != null) {
-        rb.rsp
-            .getResponseHeader()
-            .asShallowMap()
-            .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+        rb.rsp.setPartialResults();
         continue; // continue if there was an error and we're tolerant.
       }
       NamedList<NamedList<?>> secondPhaseResult =
diff --git a/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java b/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java
index 3836a234433..a099cc17ddc 100644
--- a/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java
+++ b/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java
@@ -38,6 +38,7 @@ import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.EarlyTerminatingCollectorException;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -84,6 +85,7 @@ public class SpellCheckCollator {
       IndexReader reader = ultimateResponse.req.getSearcher().getIndexReader();
       maxDocId = reader.maxDoc();
     }
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
 
     int tryNo = 0;
     int collNo = 0;
@@ -95,6 +97,10 @@ public class SpellCheckCollator {
             suggestionsMayOverlap);
     while (tryNo < maxTries && collNo < maxCollations && possibilityIter.hasNext()) {
 
+      if (queryLimits.maybeExitWithPartialResults("SpellCheck collator")) {
+        return List.of();
+      }
+
       PossibilityIterator.RankedSpellPossibility possibility = possibilityIter.next();
       String collationQueryStr = getCollation(originalQuery, possibility.corrections);
       long hits = 0;
diff --git a/solr/core/src/java/org/apache/solr/util/TestInjection.java b/solr/core/src/java/org/apache/solr/util/TestInjection.java
index 3bae33adf7d..4f1547ccd00 100644
--- a/solr/core/src/java/org/apache/solr/util/TestInjection.java
+++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java
@@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.QueryTimeout;
 import org.apache.solr.common.NonExistentCoreException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -153,6 +154,8 @@ public class TestInjection {
 
   public static volatile AtomicInteger countDocSetDelays = new AtomicInteger(0);
 
+  public static volatile QueryTimeout queryTimeout = null;
+
   public static volatile boolean failInExecutePlanAction = false;
 
   /**
@@ -199,6 +202,7 @@ public class TestInjection {
     failInExecutePlanAction = false;
     skipIndexWriterCommitOnClose = false;
     uifOutOfMemoryError = false;
+    queryTimeout = null;
     notifyPauseForeverDone();
     newSearcherHooks.clear();
     for (Timer timer : timers) {
diff --git a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
index 296b3517ff9..621a5b5b748 100644
--- a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
@@ -458,7 +458,9 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
     assertU(add(doc("id", "5")));
     assertU(commit());
 
-    // Just checking if no errors occur
+    // should exceed timeAllowed
+    // TODO: this always succeeds now, regardless of partialResults=true
+    // needs SOLR-17151 to fix how QueryLimitsExceeded exception is handled
     assertJQ(
         req(
             "q",
@@ -470,7 +472,23 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
             "group.query",
             "id:2",
             "timeAllowed",
-            "1"));
+            "1"),
+        "/responseHeader/partialResults==true");
+    // should succeed
+    assertJQ(
+        req(
+            "q",
+            "*:*",
+            "group",
+            "true",
+            "group.query",
+            "id:1",
+            "group.query",
+            "id:2",
+            "timeAllowed",
+            "200"),
+        "/grouped/id:1/matches==5",
+        "/grouped/id:2/matches==5");
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/search/CallerSpecificQueryLimit.java b/solr/core/src/test/org/apache/solr/search/CallerSpecificQueryLimit.java
new file mode 100644
index 00000000000..0d53b8653db
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/CallerSpecificQueryLimit.java
@@ -0,0 +1,105 @@
+/*
+ * 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.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.lucene.index.QueryTimeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class to simulate query timeouts at specific points in various components that call {@link
+ * QueryLimits#shouldExit()}. These calling points are identified by the calling class' simple name
+ * and optionally a method name, e.g. <code>MoreLikeThisComponent</code> or <code>
+ * ClusteringComponent.finishStage</code>.
+ */
+public class CallerSpecificQueryLimit implements QueryTimeout {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  final StackWalker stackWalker =
+      StackWalker.getInstance(StackWalker.Option.RETAIN_CLASS_REFERENCE);
+  final Map<String, Set<String>> interestingCallers = new HashMap<>();
+  public String trippedBy;
+
+  /**
+   * Signal a timeout in places that match the calling classes (and methods).
+   *
+   * @param callerExprs list of expressions in the format of <code>simpleClassName[.methodName]
+   *     </code>. If the list is empty or null then the first call to {@link #shouldExit()} from any
+   *     caller will match.
+   */
+  public CallerSpecificQueryLimit(String... callerExprs) {
+    if (callerExprs != null && callerExprs.length > 0) {
+      for (String callerExpr : callerExprs) {
+        String[] clazzMethod = callerExpr.split("\\.");
+        if (clazzMethod.length > 2) {
+          throw new RuntimeException("Invalid callerExpr: " + callerExpr);
+        }
+        Set<String> methods =
+            interestingCallers.computeIfAbsent(clazzMethod[0], c -> new HashSet<>());
+        if (clazzMethod.length > 1) {
+          methods.add(clazzMethod[1]);
+        }
+      }
+    }
+  }
+
+  @Override
+  public boolean shouldExit() {
+    Optional<String> matchingExpr =
+        stackWalker.walk(
+            s ->
+                s.filter(
+                        frame -> {
+                          Class<?> declaring = frame.getDeclaringClass();
+                          // skip bottom-most frames: myself and QueryLimits
+                          if (declaring == this.getClass() || declaring == QueryLimits.class) {
+                            return false;
+                          }
+                          String method = frame.getMethodName();
+                          if (interestingCallers.isEmpty()) {
+                            // any caller is an interesting caller
+                            return true;
+                          }
+                          Set<String> methods = interestingCallers.get(declaring.getSimpleName());
+                          if (methods == null) {
+                            return false;
+                          }
+                          return methods.isEmpty() || methods.contains(method);
+                        })
+                    .map(
+                        frame ->
+                            (frame.getDeclaringClass().getSimpleName().isBlank()
+                                    ? frame.getClassName()
+                                    : frame.getDeclaringClass().getSimpleName())
+                                + "."
+                                + frame.getMethodName())
+                    .findFirst());
+    if (matchingExpr.isPresent()) {
+      if (log.isInfoEnabled()) {
+        log.info("++++ Limit tripped by caller: {} ++++", matchingExpr.get());
+      }
+      trippedBy = matchingExpr.get();
+    }
+    return matchingExpr.isPresent();
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/search/TestQueryLimits.java b/solr/core/src/test/org/apache/solr/search/TestQueryLimits.java
new file mode 100644
index 00000000000..6e413f6bfd2
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestQueryLimits.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.search;
+
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.CloudUtil;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.ThreadCpuTimer;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestQueryLimits extends SolrCloudTestCase {
+
+  private static final String COLLECTION = "test";
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    System.setProperty(ThreadCpuTimer.ENABLE_CPU_TIME, "true");
+    configureCluster(1).addConfig("conf", configset("cloud-minimal")).configure();
+    SolrClient solrClient = cluster.getSolrClient();
+    CollectionAdminRequest.Create create =
+        CollectionAdminRequest.createCollection(COLLECTION, "conf", 3, 2);
+    create.process(solrClient);
+    CloudUtil.waitForState(
+        cluster.getOpenOverseer().getSolrCloudManager(), "active", COLLECTION, clusterShape(3, 6));
+    for (int j = 0; j < 100; j++) {
+      solrClient.add(
+          COLLECTION,
+          sdoc(
+              "id",
+              "id-" + j,
+              "val_i",
+              j % 5,
+              "text",
+              TestUtil.randomHtmlishString(random(), 100)));
+    }
+    solrClient.commit(COLLECTION);
+  }
+
+  // TODO: add more tests and better assertions once SOLR-17151 / SOLR-17158 is done
+  @Test
+  public void testQueryLimits() throws Exception {
+    SolrClient solrClient = cluster.getSolrClient();
+    QueryResponse rsp = solrClient.query(COLLECTION, params("q", "*:*"));
+    assertNull("should have full results", rsp.getHeader().get("partialResults"));
+
+    String[] matchingExprTests =
+        new String[] {
+          "SearchHandler",
+          "SearchHandler.handleRequestBody",
+          "QueryComponent",
+          "QueryComponent.process",
+          "FacetComponent.process"
+        };
+    for (String matchingExpr : matchingExprTests) {
+      CallerSpecificQueryLimit limit = new CallerSpecificQueryLimit(matchingExpr);
+      TestInjection.queryTimeout = limit;
+      rsp =
+          solrClient.query(
+              COLLECTION,
+              params("q", "id:*", "sort", "id asc", "facet", "true", "facet.field", "val_i"));
+      assertNotNull(
+          "should have partial results for expr " + matchingExpr,
+          rsp.getHeader().get("partialResults"));
+      if (matchingExpr.contains(".")) {
+        assertEquals(matchingExpr, limit.trippedBy);
+      } else {
+        assertTrue(
+            "expected result to start with " + matchingExpr + " but was " + limit.trippedBy,
+            limit.trippedBy.startsWith(matchingExpr));
+      }
+    }
+  }
+}
diff --git a/solr/modules/clustering/src/java/org/apache/solr/handler/clustering/ClusteringComponent.java b/solr/modules/clustering/src/java/org/apache/solr/handler/clustering/ClusteringComponent.java
index 09ed102b224..d18e1b7c924 100644
--- a/solr/modules/clustering/src/java/org/apache/solr/handler/clustering/ClusteringComponent.java
+++ b/solr/modules/clustering/src/java/org/apache/solr/handler/clustering/ClusteringComponent.java
@@ -54,6 +54,7 @@ import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocSlice;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.carrot2.clustering.Cluster;
@@ -290,6 +291,10 @@ public class ClusteringComponent extends SearchComponent implements SolrCoreAwar
     EngineParameters parameters = engine.defaults.derivedFrom(rb.req.getParams());
 
     List<InputDocument> inputs = getDocuments(rb, parameters);
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+    if (queryLimits.maybeExitWithPartialResults("Clustering process")) {
+      return;
+    }
 
     if (rb.req.getParams().getBool(ShardParams.IS_SHARD, false)
         && rb.req.getParams().getBool(REQUEST_PARAM_COLLECT_INPUTS, false)) {
@@ -337,7 +342,6 @@ public class ClusteringComponent extends SearchComponent implements SolrCoreAwar
                   inputs.addAll(documentsFromNamedList(partialInputs));
                 }
               });
-
       EngineEntry engine = getEngine(rb);
       EngineParameters parameters = engine.defaults.derivedFrom(rb.req.getParams());
       doCluster(rb, engine, inputs, parameters);
@@ -352,6 +356,10 @@ public class ClusteringComponent extends SearchComponent implements SolrCoreAwar
       EngineParameters parameters) {
     // log.warn("# CLUSTERING: " + inputs.size() + " document(s), contents:\n - "
     //   + inputs.stream().map(Object::toString).collect(Collectors.joining("\n - ")));
+    QueryLimits queryLimits = QueryLimits.getCurrentLimits();
+    if (queryLimits.maybeExitWithPartialResults("Clustering doCluster")) {
+      return;
+    }
     List<Cluster<InputDocument>> clusters = engine.get().cluster(parameters, rb.getQuery(), inputs);
     rb.rsp.add(RESPONSE_SECTION_CLUSTERS, clustersToNamedList(inputs, clusters, parameters));
   }
diff --git a/solr/solr-ref-guide/modules/query-guide/pages/common-query-parameters.adoc b/solr/solr-ref-guide/modules/query-guide/pages/common-query-parameters.adoc
index 9498176edb9..201c6c86a24 100644
--- a/solr/solr-ref-guide/modules/query-guide/pages/common-query-parameters.adoc
+++ b/solr/solr-ref-guide/modules/query-guide/pages/common-query-parameters.adoc
@@ -284,6 +284,14 @@ The query above allows you to examine the scoring explain info of the top matchi
 
 The default value of this parameter is blank, which causes no extra "explain info" to be returned.
 
+== partialResults Parameter
+
+This parameter controls Solr's behavior when a query execution limit is reached (e.g. `timeAllowed` or `cpuAllowed`).
+
+When this parameter is set to `true` (default) then even though reaching a limit terminates further query processing  Solr will still attempt to return partial results collected so far. These results may be incomplete in a non-deterministic way (e.g. only some matching documents, documents without fields, missing facets or pivots, no spellcheck results, etc).
+
+When this parameter is set to `false` then reaching a limit will generate an exception and any partial results collected so far will be discarded.
+
 == timeAllowed Parameter
 
 This parameter specifies the amount of time, in milliseconds, allowed for a search to complete.
@@ -325,17 +333,11 @@ Regular search, JSON Facet and the Analytics component abandon requests in accor
 == cpuAllowed Parameter
 
 This parameter specifies the amount of CPU time, in milliseconds, allowed for a search to complete.
-In contrast to the `timeAllowed` this parameter monitors the actual CPU usage by the thread that
-executes the query. The same CPU usage limit is applied to the query coordinator as to each replica
-that participates in the distributed search (although reaching this limit first in the query coordinator is unlikely).
-Should any replica locally exceed the allowed CPU time the whole distributed search will be terminated
-(by canceling requests to other shards).
-
-Note: the same CPU limit is applied to each stage in the distributed query processing. Typically this
-involves two or more stages (e.g. getting top document id-s, retrieving their fields, additional stages may be
-required for faceting, grouping, etc). For example, setting `cpuAllowed=500` gives a limit of at most
-500 ms of CPU time for each of these stages - meaning that the total CPU usage by the query may reach a multiple
-of the `cpuAllowed` value depending on the number of stages.
+In contrast to the `timeAllowed` this parameter monitors the actual CPU usage by the thread that executes the query. The same CPU usage limit is applied to the query coordinator as to each replica that participates in the distributed search (although reaching this limit first in the query coordinator is unlikely).
+Should any replica locally exceed the allowed CPU time the whole distributed search will be terminated (by canceling requests to other shards).
+
+Note: the same CPU limit is applied to each stage in the distributed query processing. Typically this involves two or more stages (e.g. getting top document id-s, retrieving their fields, additional stages may be required for faceting, grouping, etc).
+For example, setting `cpuAllowed=500` gives a limit of at most 500 ms of CPU time for each of these stages - meaning that the total CPU usage by the query may reach a multiple of the `cpuAllowed` value depending on the number of stages.
 
 All other considerations regarding partial results listed for the `timeAllowed` parameter apply here, too.
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrQuery.java b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrQuery.java
index 11bd3cf3f7c..5719efcd180 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrQuery.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrQuery.java
@@ -1198,6 +1198,29 @@ public class SolrQuery extends ModifiableSolrParams {
     return this.getInt(CommonParams.TIME_ALLOWED);
   }
 
+  /**
+   * Set the maximum CPU time allowed for this query. If the thread that executes the query takes
+   * more CPU time than the specified milliseconds, a timeout occurs and partial (or no) results may
+   * be returned.
+   *
+   * <p>If given Integer is null, then this parameter is removed from the request
+   *
+   * @param milliseconds the CPU time in milliseconds allowed for this query
+   */
+  public SolrQuery setCpuAllowed(Integer milliseconds) {
+    if (milliseconds == null) {
+      this.remove(CommonParams.CPU_ALLOWED);
+    } else {
+      this.set(CommonParams.CPU_ALLOWED, milliseconds);
+    }
+    return this;
+  }
+
+  /** Get the maximum time allowed for this query. */
+  public Integer getCpuAllowed() {
+    return this.getInt(CommonParams.CPU_ALLOWED);
+  }
+
   ///////////////////////
   //  Utility functions
   ///////////////////////
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
index cc9e07e959d..9fca76238b3 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
@@ -158,6 +158,12 @@ public interface CommonParams {
 
   boolean SEGMENT_TERMINATE_EARLY_DEFAULT = false;
 
+  /**
+   * If true then allow returning partial results. If false and full results can't be produced
+   * return no results / error.
+   */
+  String PARTIAL_RESULTS = "partialResults";
+
   /** Timeout value in milliseconds. If not set, or the value is &lt; 0, there is no timeout. */
   String TIME_ALLOWED = "timeAllowed";