From b748207bc331b5eeae284ee7602626dbe5e3ff50 Mon Sep 17 00:00:00 2001 From: Gus Heck <46900717+gus-asf@users.noreply.github.com> Date: Tue, 1 Oct 2024 20:12:21 -0400 Subject: [PATCH] SOLR-17158 Terminate distributed processing faster when query limit is reached and partial results are not needed (#2666) --- solr/CHANGES.txt | 6 + .../solr/handler/RequestHandlerBase.java | 4 +- .../handler/component/HttpShardHandler.java | 234 +++++++++++++++--- .../component/ParallelHttpShardHandler.java | 97 +++++--- .../handler/component/QueryComponent.java | 7 +- .../handler/component/ResponseBuilder.java | 4 +- .../solr/handler/component/SearchHandler.java | 20 +- .../solr/handler/component/ShardHandler.java | 10 +- .../solr/handler/component/ShardResponse.java | 7 +- .../apache/solr/request/SolrQueryRequest.java | 35 +++ .../apache/solr/request/SolrRequestInfo.java | 9 + .../solr/response/SolrQueryResponse.java | 58 ++++- .../org/apache/solr/search/QueryLimits.java | 19 +- .../org/apache/solr/search/QueryResult.java | 12 +- .../apache/solr/search/facet/FacetModule.java | 8 +- .../solr/search/grouping/CommandHandler.java | 16 +- .../SearchGroupShardResponseProcessor.java | 5 +- .../TopGroupsShardResponseProcessor.java | 5 +- .../solr/search/stats/ExactStatsCache.java | 4 +- .../apache/solr/util/SolrResponseUtil.java | 4 +- .../solr/search/TestCpuAllowedLimit.java | 24 +- .../pages/common-query-parameters.adoc | 15 +- .../pages/major-changes-in-solr-9.adoc | 6 + .../solr/common/params/ShardParams.java | 26 +- .../solr/common/params/ShardParamsTest.java | 15 +- 25 files changed, 490 insertions(+), 160 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index bd9cf256425..8daccd2078b 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -111,6 +111,12 @@ New Features Improvements --------------------- +* SOLR-17158: Users using query limits (timeAllowed, cpuTimeAllowed) for whom partial results are uninteresting + may set partialResults=false. This parameter has been enhanced to reduce time spent processing partial results + and omit partialResults from the response. Since this is requested behavior, no exception is thrown and the + partialResults response header will always exist if the result was short circuited. + (Gus Heck, Andrzej Bialecki, hossman) + * SOLR-17397: SkipExistingDocumentsProcessor now functions correctly with child documents. (Tim Owens via Eric Pugh) * SOLR-17180: Deprecate snapshotscli.sh in favour of bin/solr snapshot sub commands. Now able to manage Snapshots from the CLI. HDFS module specific snapshot script now ships as part of that module in the modules/hdfs/bin directory. (Eric Pugh) 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 d85b939fbb5..9f438fc4605 100644 --- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java +++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java @@ -17,6 +17,7 @@ package org.apache.solr.handler; import static org.apache.solr.core.RequestParams.USEPARAM; +import static org.apache.solr.response.SolrQueryResponse.haveCompleteResults; import com.codahale.metrics.Counter; import com.codahale.metrics.Meter; @@ -235,7 +236,8 @@ public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) { rsp.setHttpCaching(httpCaching); handleRequestBody(req, rsp); // count timeouts - if (rsp.isPartialResults()) { + + if (!haveCompleteResults(rsp.getResponseHeader())) { metrics.numTimeouts.mark(); rsp.setHttpCaching(false); } diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java index 3bc1c542906..219197d8cd8 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java @@ -16,6 +16,9 @@ */ package org.apache.solr.handler.component; +import static org.apache.solr.common.params.CommonParams.PARTIAL_RESULTS; +import static org.apache.solr.request.SolrQueryRequest.disallowPartialResults; + import java.util.HashMap; import java.util.List; import java.util.Map; @@ -24,6 +27,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import net.jcip.annotations.NotThreadSafe; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrResponse; @@ -42,6 +46,7 @@ import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.NamedList; +import org.apache.solr.common.util.StrUtils; import org.apache.solr.core.CoreDescriptor; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrRequestInfo; @@ -60,6 +65,7 @@ */ @NotThreadSafe public class HttpShardHandler extends ShardHandler { + /** * If the request context map has an entry with this key and Boolean.TRUE as value, {@link * #prepDistributed(ResponseBuilder)} will only include {@link @@ -70,11 +76,43 @@ public class HttpShardHandler extends ShardHandler { */ public static String ONLY_NRT_REPLICAS = "distribOnlyRealtime"; - protected HttpShardHandlerFactory httpShardHandlerFactory; + private final HttpShardHandlerFactory httpShardHandlerFactory; + + /* + * Three critical fields: + * - pending: keeps track of how many things we started + * - responseFutureMap: holds futures for anything not yet complete + * - responses: the result of things we started, when responses.size() + * + * All of this must be kept consistent and is therefore synchronized on RESPONSES_LOCK + * The exception is when a response is added so long as pending is incremented first + * because responses is a LinkedBlockingQueue and that is synchronized. The response + * future map is not synchronized however, and so we need to guard it for both order + * and memory consistency (happens before) reasons. + * + * The code works by looping/decrementing pending until responses.size() matches the + * size of the shard list. Thus, there is a tricky, hidden assumption of one response + * for every shard, even if the shard is down (so we add a fake response with a shard + * down exception). Note that down shards have a shard url of empty string in this case. + * + * This seems overcomplicated. Perhaps this can someday be changed to simply + * test responses.size == pending.size? + */ + protected final Object FUTURE_MAP_LOCK = new Object(); + protected Map> responseFutureMap; protected BlockingQueue responses; + + /** + * The number of pending requests. This must be incremented before a {@link ShardResponse} is + * added to {@link #responses}, and decremented after a ShardResponse is removed from {@code + * responses}. We can't rely on responseFutureMap.size() because it is an unsynchronized + * collection updated by multiple threads, and it's internal state including the size field is not + * volatile/synchronized. + */ protected AtomicInteger pending; - protected Map> shardToURLs; + + private final Map> shardToURLs; protected LBHttp2SolrClient lbClient; public HttpShardHandler(HttpShardHandlerFactory httpShardHandlerFactory) { @@ -91,6 +129,39 @@ public HttpShardHandler(HttpShardHandlerFactory httpShardHandlerFactory) { shardToURLs = new HashMap<>(); } + /** + * Parse the {@value ShardParams#SHARDS_TOLERANT} param from params as a boolean; + * accepts {@value ShardParams#REQUIRE_ZK_CONNECTED} as a valid value indicating false + * . + * + *

By default, returns false when {@value ShardParams#SHARDS_TOLERANT} is not set + * in + * params. + */ + public static boolean getShardsTolerantAsBool(SolrQueryRequest req) { + String shardsTolerantValue = req.getParams().get(ShardParams.SHARDS_TOLERANT); + if (null == shardsTolerantValue + || shardsTolerantValue.trim().equals(ShardParams.REQUIRE_ZK_CONNECTED)) { + return false; + } else { + boolean tolerant = StrUtils.parseBool(shardsTolerantValue.trim()); + if (tolerant && disallowPartialResults(req.getParams())) { + throw new SolrException( + SolrException.ErrorCode.BAD_REQUEST, + "Use of " + + ShardParams.SHARDS_TOLERANT + + " requires that " + + PARTIAL_RESULTS + + " is true. If " + + PARTIAL_RESULTS + + " is defaulted to false explicitly passing " + + PARTIAL_RESULTS + + "=true in the request will allow shards.tolerant to work"); + } + return tolerant; // throw an exception if non-boolean + } + } + public static class SimpleSolrResponse extends SolrResponse { volatile long elapsedTime; @@ -120,7 +191,7 @@ public void setElapsedTime(long elapsedTime) { // Not thread safe... don't use in Callable. // Don't modify the returned URL list. - protected List getURLs(String shard) { + private List getURLs(String shard) { List urls = shardToURLs.get(shard); if (urls == null) { urls = httpShardHandlerFactory.buildURLList(shard); @@ -129,11 +200,11 @@ protected List getURLs(String shard) { return urls; } - protected LBSolrClient.Req prepareLBRequest( + private LBSolrClient.Req prepareLBRequest( ShardRequest sreq, String shard, ModifiableSolrParams params, List urls) { params.remove(CommonParams.WT); // use default (currently javabin) params.remove(CommonParams.VERSION); - QueryRequest req = makeQueryRequest(sreq, params, shard); + QueryRequest req = createQueryRequest(sreq, params, shard); req.setMethod(SolrRequest.METHOD.POST); SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo(); if (requestInfo != null) { @@ -143,7 +214,7 @@ protected LBSolrClient.Req prepareLBRequest( return httpShardHandlerFactory.newLBHttpSolrClientReq(req, urls); } - protected ShardResponse prepareShardResponse(ShardRequest sreq, String shard) { + private ShardResponse prepareShardResponse(ShardRequest sreq, String shard) { ShardResponse srsp = new ShardResponse(); if (sreq.nodeName != null) { srsp.setNodeName(sreq.nodeName); @@ -154,14 +225,19 @@ protected ShardResponse prepareShardResponse(ShardRequest sreq, String shard) { return srsp; } - protected void recordNoUrlShardResponse(ShardResponse srsp, String shard) { + private void recordNoUrlShardResponse(ShardResponse srsp, String shard) { // TODO: what's the right error code here? We should use the same thing when // all of the servers for a shard are down. + // TODO: shard is a blank string in this case, which is somewhat less than helpful SolrException exception = new SolrException( SolrException.ErrorCode.SERVICE_UNAVAILABLE, "no servers hosting shard: " + shard); srsp.setException(exception); srsp.setResponseCode(exception.code()); + + // order of next two statements is important. Both are synchronized objects so + // synchronization is needed so long as the order is correct. + pending.incrementAndGet(); responses.add(srsp); } @@ -173,46 +249,60 @@ public void submit(ShardRequest sreq, String shard, ModifiableSolrParams params) final var srsp = prepareShardResponse(sreq, shard); final var ssr = new SimpleSolrResponse(); srsp.setSolrResponse(ssr); - pending.incrementAndGet(); - if (urls.isEmpty()) { recordNoUrlShardResponse(srsp, shard); return; } + long startTimeNS = System.nanoTime(); - long startTime = System.nanoTime(); - CompletableFuture future = this.lbClient.requestAsync(lbReq); - future.whenComplete( - (rsp, throwable) -> { - if (rsp != null) { - ssr.nl = rsp.getResponse(); - srsp.setShardAddress(rsp.getServer()); - ssr.elapsedTime = - TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS); - responses.add(transfomResponse(sreq, srsp, shard)); - } else if (throwable != null) { - ssr.elapsedTime = - TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS); - srsp.setException(throwable); - if (throwable instanceof SolrException) { - srsp.setResponseCode(((SolrException) throwable).code()); - } - responses.add(transfomResponse(sreq, srsp, shard)); - } - }); + makeShardRequest(sreq, shard, params, lbReq, ssr, srsp, startTimeNS); + } - responseFutureMap.put(srsp, future); + /** + * Do the actual work of sending a request to a shard and receiving the response + * + * @param sreq the request to make + * @param shard the shard to address + * @param params request parameters + * @param lbReq the load balanced request suitable for LBHttp2SolrClient + * @param ssr the response collector part 1 + * @param srsp the shard response collector + * @param startTimeNS the time at which the request was initiated, likely just prior to calling + * this method. + */ + // future work might see if we can reduce this parameter list. For example, + // why do we need 2 separate response objects? + protected void makeShardRequest( + ShardRequest sreq, + String shard, + ModifiableSolrParams params, + LBSolrClient.Req lbReq, + SimpleSolrResponse ssr, + ShardResponse srsp, + long startTimeNS) { + CompletableFuture future = this.lbClient.requestAsync(lbReq); + future.whenComplete(new ShardRequestCallback(ssr, srsp, startTimeNS, sreq, shard, params)); + synchronized (FUTURE_MAP_LOCK) { + // we want to ensure that there is a future in flight before incrementing + // pending. If anything fails such that a request/future is not created there is + // potential for the request to hang forever waiting on a responses.take() + // and so if anything failed during future creation we would get stuck. + pending.incrementAndGet(); + responseFutureMap.put(srsp, future); + } } /** Subclasses could modify the request based on the shard */ - protected QueryRequest makeQueryRequest( + @SuppressWarnings("unused") + protected QueryRequest createQueryRequest( final ShardRequest sreq, ModifiableSolrParams params, String shard) { // use generic request to avoid extra processing of queries return new QueryRequest(params); } /** Subclasses could modify the Response based on the shard */ - protected ShardResponse transfomResponse( + @SuppressWarnings("unused") + protected ShardResponse transformResponse( final ShardRequest sreq, ShardResponse rsp, String shard) { return rsp; } @@ -229,11 +319,19 @@ public ShardResponse takeCompletedOrError() { private ShardResponse take(boolean bailOnError) { try { - while (pending.get() > 0) { - ShardResponse rsp = responses.take(); - responseFutureMap.remove(rsp); + while (responsesPending()) { + ShardResponse rsp; + synchronized (FUTURE_MAP_LOCK) { + // in the parallel case we need to recheck responsesPending() + // in case all attempts to submit failed. + rsp = responses.poll(50, TimeUnit.MILLISECONDS); + if (rsp == null) { + continue; + } + responseFutureMap.remove(rsp); + pending.decrementAndGet(); + } - pending.decrementAndGet(); if (bailOnError && rsp.getException() != null) return rsp; // if exception, return immediately // add response to the response list... we do this after the take() and @@ -241,6 +339,7 @@ private ShardResponse take(boolean bailOnError) { // for a request was received. Otherwise we might return the same // request more than once. rsp.getShardRequest().responses.add(rsp); + if (rsp.getShardRequest().responses.size() == rsp.getShardRequest().actualShards.length) { return rsp; } @@ -251,13 +350,19 @@ private ShardResponse take(boolean bailOnError) { return null; } + protected boolean responsesPending() { + return pending.get() > 0; + } + @Override public void cancelAll() { - for (CompletableFuture future : responseFutureMap.values()) { - future.cancel(true); - pending.decrementAndGet(); + synchronized (FUTURE_MAP_LOCK) { + for (CompletableFuture future : responseFutureMap.values()) { + future.cancel(true); + pending.decrementAndGet(); + } + responseFutureMap.clear(); } - responseFutureMap.clear(); } @Override @@ -312,7 +417,7 @@ public void prepDistributed(ResponseBuilder rb) { // be an optimization? } - if (!ShardParams.getShardsTolerantAsBool(params)) { + if (!getShardsTolerantAsBool(req)) { for (int i = 0; i < rb.slices.length; i++) { if (replicaSource.getReplicasBySlice(i).isEmpty()) { final ReplicaSource allActiveReplicaSource = @@ -395,4 +500,51 @@ private boolean canShortCircuit( public ShardHandlerFactory getShardHandlerFactory() { return httpShardHandlerFactory; } + + class ShardRequestCallback implements BiConsumer { + private final SimpleSolrResponse ssr; + private final ShardResponse srsp; + private final long startTimeNS; + private final ShardRequest sreq; + private final String shard; + private final ModifiableSolrParams params; + + public ShardRequestCallback( + SimpleSolrResponse ssr, + ShardResponse srsp, + long startTimeNS, + ShardRequest sreq, + String shard, + ModifiableSolrParams params) { + this.ssr = ssr; + this.srsp = srsp; + this.startTimeNS = startTimeNS; + this.sreq = sreq; + this.shard = shard; + this.params = params; + } + + @Override + public void accept(LBSolrClient.Rsp rsp, Throwable throwable) { + if (rsp != null) { + ssr.nl = rsp.getResponse(); + srsp.setShardAddress(rsp.getServer()); + ssr.elapsedTime = + TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTimeNS, TimeUnit.NANOSECONDS); + responses.add(HttpShardHandler.this.transformResponse(sreq, srsp, shard)); + } else if (throwable != null) { + ssr.elapsedTime = + TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTimeNS, TimeUnit.NANOSECONDS); + srsp.setException(throwable); + if (throwable instanceof SolrException) { + srsp.setResponseCode(((SolrException) throwable).code()); + } + responses.add(HttpShardHandler.this.transformResponse(sreq, srsp, shard)); + if (disallowPartialResults(params)) { + HttpShardHandler.this + .cancelAll(); // Note: method synchronizes RESPONSE_CANCELABLE_LOCK on entry + } + } + } + } } diff --git a/solr/core/src/java/org/apache/solr/handler/component/ParallelHttpShardHandler.java b/solr/core/src/java/org/apache/solr/handler/component/ParallelHttpShardHandler.java index 4a8dd5e3267..7d6cf13874f 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/ParallelHttpShardHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/component/ParallelHttpShardHandler.java @@ -17,13 +17,11 @@ package org.apache.solr.handler.component; import java.lang.invoke.MethodHandles; -import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import net.jcip.annotations.NotThreadSafe; import org.apache.solr.client.solrj.impl.LBSolrClient; -import org.apache.solr.common.SolrException; import org.apache.solr.common.params.ModifiableSolrParams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,57 +39,78 @@ @NotThreadSafe public class ParallelHttpShardHandler extends HttpShardHandler { + @SuppressWarnings("unused") private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private final ExecutorService commExecutor; + /* + * Unlike the basic HttpShardHandler, this class allows us to exit submit before + * pending is incremented and the responseFutureMap is updated. If the runnables that + * do that are slow to execute the calling code could attempt to takeCompleted(), + * while pending is still zero. In this condition, the code would assume that all + * requests are processed (despite the runnables created by this class still + * waiting). Thus, we need to track that there are attempts still in flight. + * + * This tracking is complicated by the fact that there could be a failure in the + * runnable that causes the request to never be created and pending to never be + * incremented. Thus, we need to know that we have attempted something AND that that + * attempt has also been processed by the executor. + * + * This condition is added to the check that controls the loop in take via the + * override for #responsesPending(). We rely on calling code call submit for all + * requests desired before the call to takeCompleted() + */ + AtomicInteger attemptStart = new AtomicInteger(0); + AtomicInteger attemptCount = new AtomicInteger(0); + public ParallelHttpShardHandler(ParallelHttpShardHandlerFactory httpShardHandlerFactory) { super(httpShardHandlerFactory); this.commExecutor = httpShardHandlerFactory.commExecutor; } @Override - public void submit(ShardRequest sreq, String shard, ModifiableSolrParams params) { - // do this outside of the callable for thread safety reasons - final List urls = getURLs(shard); - final var lbReq = prepareLBRequest(sreq, shard, params, urls); - final var srsp = prepareShardResponse(sreq, shard); - final var ssr = new SimpleSolrResponse(); - srsp.setSolrResponse(ssr); - pending.incrementAndGet(); - - if (urls.isEmpty()) { - recordNoUrlShardResponse(srsp, shard); - return; - } + protected boolean responsesPending() { + // ensure we can't exit while loop in HttpShardHandler.take(boolean) until we've completed + // as many Runnable actions as we created. + return super.responsesPending() || attemptStart.get() > attemptCount.get(); + } - long startTime = System.nanoTime(); + @Override + protected void makeShardRequest( + ShardRequest sreq, + String shard, + ModifiableSolrParams params, + LBSolrClient.Req lbReq, + SimpleSolrResponse ssr, + ShardResponse srsp, + long startTimeNS) { final Runnable executeRequestRunnable = () -> { - CompletableFuture future = this.lbClient.requestAsync(lbReq); - future.whenComplete( - (rsp, throwable) -> { - if (rsp != null) { - ssr.nl = rsp.getResponse(); - srsp.setShardAddress(rsp.getServer()); - ssr.elapsedTime = - TimeUnit.MILLISECONDS.convert( - System.nanoTime() - startTime, TimeUnit.NANOSECONDS); - responses.add(srsp); - } else if (throwable != null) { - ssr.elapsedTime = - TimeUnit.MILLISECONDS.convert( - System.nanoTime() - startTime, TimeUnit.NANOSECONDS); - srsp.setException(throwable); - if (throwable instanceof SolrException) { - srsp.setResponseCode(((SolrException) throwable).code()); - } - responses.add(srsp); - } - }); - responseFutureMap.put(srsp, future); + try { + CompletableFuture future = this.lbClient.requestAsync(lbReq); + future.whenComplete( + new ShardRequestCallback(ssr, srsp, startTimeNS, sreq, shard, params)); + synchronized (FUTURE_MAP_LOCK) { + // we want to ensure that there is a future in flight before incrementing + // pending, because there is a risk that the request will hang forever waiting + // on a responses.take() in HttpShardHandler.take(boolean) if anything failed + // during future creation. It is not a problem if the response shows up before + // we increment pending. The attemptingSubmit flag guards us against inadvertently + // skipping the while loop in HttpShardHandler.take(boolean) until at least + // one runnable has been executed. + pending.incrementAndGet(); + responseFutureMap.put(srsp, future); + } + } finally { + // it must not be possible to exit the runnable in any way without calling this. + attemptCount.incrementAndGet(); + } }; + // not clear how errors emanating from requestAsync or the whenComplete() callback + // are to propagated out of the runnable? + attemptStart.incrementAndGet(); CompletableFuture.runAsync(executeRequestRunnable, commExecutor); } } 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 3c558feffc3..3acb78ed78f 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 @@ -17,6 +17,7 @@ package org.apache.solr.handler.component; import static org.apache.solr.common.params.CommonParams.QUERY_UUID; +import static org.apache.solr.response.SolrQueryResponse.haveCompleteResults; import java.io.IOException; import java.io.PrintWriter; @@ -1338,10 +1339,8 @@ protected void returnFields(ResponseBuilder rb, ShardRequest sreq) { (NamedList) SolrResponseUtil.getSubsectionFromShardResponse( rb, srsp, "responseHeader", false)); - if (Boolean.TRUE.equals( - responseHeader.getBooleanArg( - SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY))) { - rb.rsp.setPartialResults(); + if (!haveCompleteResults(responseHeader)) { // partial or omitted partials + rb.rsp.setPartialResults(rb.req); rb.rsp.addPartialResponseDetail( responseHeader.get(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_DETAILS_KEY)); } 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 dc58a540d1d..55addf143f8 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 @@ -444,8 +444,10 @@ public Query wrap(Query q) { /** Sets results from a SolrIndexSearcher.QueryResult. */ public void setResult(QueryResult result) { setResults(result.getDocListAndSet()); + if (result.isPartialResultOmitted() || result.isPartialResults()) { + rsp.setPartialResults(req); + } if (result.isPartialResults()) { - 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 fd7bec182c5..28db4f4478c 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 @@ -488,7 +488,7 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw debug.add("explain", new NamedList<>()); rb.rsp.add("debug", debug); } - rb.rsp.setPartialResults(); + rb.rsp.setPartialResults(rb.req); } } else { // a distributed request @@ -553,7 +553,7 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw // now wait for replies, but if anyone puts more requests on // the outgoing queue, send them out immediately (by exiting // this loop) - boolean tolerant = ShardParams.getShardsTolerantAsBool(rb.req.getParams()); + boolean tolerant = HttpShardHandler.getShardsTolerantAsBool(rb.req); while (rb.outgoing.size() == 0) { ShardResponse srsp = tolerant @@ -561,6 +561,20 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw : shardHandler1.takeCompletedOrError(); if (srsp == null) break; // no more requests to wait for + boolean anyResponsesPartial = + srsp.getShardRequest().responses.stream() + .anyMatch( + response -> { + NamedList resp = response.getSolrResponse().getResponse(); + if (resp == null) { + return false; + } + Object recursive = resp.findRecursive("responseHeader", "partialResults"); + return recursive != null; + }); + if (anyResponsesPartial) { + rsp.setPartialResults(rb.req); + } // Was there an exception? if (srsp.getException() != null) { // If things are not tolerant, abort everything and rethrow @@ -581,7 +595,7 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw if (allShardsFailed) { throwSolrException(srsp.getException()); } else { - rsp.setPartialResults(); + rsp.setPartialResults(rb.req); } } } diff --git a/solr/core/src/java/org/apache/solr/handler/component/ShardHandler.java b/solr/core/src/java/org/apache/solr/handler/component/ShardHandler.java index 2717bc47845..1246b2e2484 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/ShardHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/component/ShardHandler.java @@ -55,8 +55,9 @@ public abstract class ShardHandler { public abstract void submit(ShardRequest sreq, String shard, ModifiableSolrParams params); /** - * returns a ShardResponse of the last response correlated with a ShardRequest. This won't return - * early if it runs into an error. + * Returns a ShardResponse of the last response correlated with a ShardRequest. This won't return + * early if it runs into an error. Callers are responsible for ensuring that this can't be called + * before requests have been submitted with submit. */ public abstract ShardResponse takeCompletedIncludingErrors(); @@ -64,8 +65,9 @@ public abstract class ShardHandler { // distinguish between different ShardRequest objects as it seems to advertise? What's going on // here? /** - * returns a ShardResponse of the last response correlated with a ShardRequest, or immediately - * returns a ShardResponse if there was an error detected + * Returns a ShardResponse of the last response correlated with a ShardRequest, or immediately + * returns a ShardResponse if there was an error detected. Callers are responsible for ensuring + * that this can't be called before requests have been submitted with submit. */ public abstract ShardResponse takeCompletedOrError(); diff --git a/solr/core/src/java/org/apache/solr/handler/component/ShardResponse.java b/solr/core/src/java/org/apache/solr/handler/component/ShardResponse.java index dee5fcdb803..289a6e94182 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/ShardResponse.java +++ b/solr/core/src/java/org/apache/solr/handler/component/ShardResponse.java @@ -19,11 +19,16 @@ import org.apache.solr.client.solrj.SolrResponse; public final class ShardResponse { + + // WARNING: this class is used as a key in a map and is modified during that period. + // Do not implement equals() or hashCode() without considering impact on + // HttpShardHandler.responseFutureMap + private ShardRequest req; private String shard; private String nodeName; private volatile String shardAddress; // the specific shard that this response was received from - private int rspCode; + private volatile int rspCode; private volatile Throwable exception; private SolrResponse rsp; diff --git a/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java b/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java index 113aeff6440..ab020ff2509 100644 --- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java +++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java @@ -22,9 +22,11 @@ import java.util.List; import java.util.Map; import org.apache.solr.cloud.CloudDescriptor; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.CommandOperation; import org.apache.solr.common.util.ContentStream; +import org.apache.solr.common.util.EnvUtils; import org.apache.solr.core.CoreContainer; import org.apache.solr.core.SolrCore; import org.apache.solr.schema.IndexSchema; @@ -39,6 +41,39 @@ */ public interface SolrQueryRequest extends AutoCloseable { + /** This is the system property for {@link #ALLOW_PARTIAL_RESULTS_DEFAULT} */ + String SOLR_ALLOW_PARTIAL_RESULTS_DEFAULT = "solr.allowPartialResultsDefault"; + + // silly getBoolean doesn't take a default. + /** + * Users can set {@link SolrQueryRequest#SOLR_ALLOW_PARTIAL_RESULTS_DEFAULT} system property to + * true, and solr will omit results when any shard fails due query execution limits (time, cpu + * etc.). By default, this is set to true. Setting it to false will reduce processing, cpu and + * network associated with collecting and transmitting partial results. This setting can be + * overridden (in either direction) on a per-request basis with {@code + * &allowPartialResults=[true|false]}. When results have been omitted the response header should + * contain a partialResults element with the value "omitted" + */ + boolean ALLOW_PARTIAL_RESULTS_DEFAULT = + EnvUtils.getPropertyAsBool(SOLR_ALLOW_PARTIAL_RESULTS_DEFAULT, true); + + /** + * Tests if the partials for the request should be discarded. Examines {@link + * SolrQueryRequest#ALLOW_PARTIAL_RESULTS_DEFAULT} system property and also examines {@link + * CommonParams#PARTIAL_RESULTS} request param. The Request Parameter takes precedence if both are + * set. + * + * @return true if partials should be discarded. + * @param params the request parameters + */ + static boolean allowPartialResults(SolrParams params) { + return params.getBool(CommonParams.PARTIAL_RESULTS, ALLOW_PARTIAL_RESULTS_DEFAULT); + } + + static boolean disallowPartialResults(SolrParams params) { + return !allowPartialResults(params); + } + /** returns the current request parameters */ SolrParams getParams(); 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 193b838e9cd..61b51a6af20 100644 --- a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java +++ b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java @@ -24,6 +24,7 @@ import java.util.Date; import java.util.Deque; import java.util.List; +import java.util.Optional; import java.util.TimeZone; import java.util.concurrent.atomic.AtomicReference; import javax.servlet.http.HttpServletRequest; @@ -66,6 +67,14 @@ public static SolrRequestInfo getRequestInfo() { return stack.peek(); } + public static Optional getReqInfo() { + return Optional.ofNullable(getRequestInfo()); + } + + public static Optional getRequest() { + return getReqInfo().map(i -> i.req); + } + /** * Adds the SolrRequestInfo onto a stack held in a {@link ThreadLocal}. Remember to call {@link * #clearRequestInfo()}! 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 2f952649fd0..b399ce57b79 100644 --- a/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java +++ b/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java @@ -16,15 +16,18 @@ */ package org.apache.solr.response; +import static org.apache.solr.request.SolrQueryRequest.disallowPartialResults; + import java.util.Collection; import java.util.Date; 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.SolrException; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; +import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.search.ReturnFields; import org.apache.solr.search.SolrReturnFields; @@ -61,7 +64,7 @@ */ public class SolrQueryResponse { public static final String NAME = "response"; - public static final String RESPONSE_HEADER_PARTIAL_RESULTS_KEY = CommonParams.PARTIAL_RESULTS; + public static final String RESPONSE_HEADER_PARTIAL_RESULTS_KEY = "partialResults"; public static final String RESPONSE_HEADER_PARTIAL_RESULTS_DETAILS_KEY = "partialResultsDetails"; public static final String RESPONSE_HEADER_SEGMENT_TERMINATED_EARLY_KEY = "segmentTerminatedEarly"; @@ -139,30 +142,63 @@ public ReturnFields getReturnFields() { /** * If {@link #getResponseHeader()} is available, set {@link #RESPONSE_HEADER_PARTIAL_RESULTS_KEY} - * flag to true. + * attribute to true or "omitted" as required. */ - public void setPartialResults() { + public void setPartialResults(SolrQueryRequest req) { NamedList 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 (header != null) { + if (header.get(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY) == null) { + Object value = partialResultsStatus(disallowPartialResults(req.getParams())); + header.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, value); + } } } + public static Object partialResultsStatus(boolean discarding) { + return discarding ? "omitted" : Boolean.TRUE; + } + /** * If {@link #getResponseHeader()} is available, return the value of {@link * #RESPONSE_HEADER_PARTIAL_RESULTS_KEY} or false. + * + * @param header the response header + * @return true if there are results, but they do not represent the full results, false if the + * results are complete, or intentionally omitted */ - public boolean isPartialResults() { - NamedList header = getResponseHeader(); + public static boolean isPartialResults(NamedList header) { if (header != null) { - return Boolean.TRUE.equals( - header.getBooleanArg(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY)); + // actual value may be true/false/omitted + return "true" + .equalsIgnoreCase(String.valueOf(header.get(RESPONSE_HEADER_PARTIAL_RESULTS_KEY))); } else { return false; } } + /** + * Test if the entire results have been returned, or if some form of limit/cancel/tolerant logic + * has returned incomplete (or possibly empty) results. + * + * @param header the response header + * @return true if full results are returning normally false otherwise. + */ + public static boolean haveCompleteResults(NamedList header) { + if (header == null) { + // partial/omitted results will have placed something in the header, so it should exist. + return true; + } + // "true" and "omitted" should both respond with false + Object o = header.get(RESPONSE_HEADER_PARTIAL_RESULTS_KEY); + // putting this check here so that if anything new is added we don't forget to consider the + // effect on code that calls this function. Could contain either Boolean.TRUE or "omitted" + if (o != null && !(Boolean.TRUE.equals(o) || "omitted".equals(o))) { + throw new SolrException( + SolrException.ErrorCode.SERVER_ERROR, "Unrecognized value for partialResults:" + o); + } + return o == null; + } + /** * If {@link #getResponseHeader()} is available, add a reason for returning partial response. * 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 86c7f488de3..bf4dc2d08b8 100644 --- a/solr/core/src/java/org/apache/solr/search/QueryLimits.java +++ b/solr/core/src/java/org/apache/solr/search/QueryLimits.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Optional; import org.apache.lucene.index.QueryTimeout; +import org.apache.solr.common.SolrException; import org.apache.solr.common.params.CommonParams; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrRequestInfo; @@ -35,6 +36,7 @@ * return true the next time it is checked (it may be checked in either Lucene code or Solr code) */ public class QueryLimits implements QueryTimeout { + public static final String UNLIMITED = "This request is unlimited."; private final List limits = new ArrayList<>(3); // timeAllowed, cpu, and memory anticipated @@ -59,8 +61,7 @@ private QueryLimits() { */ public QueryLimits(SolrQueryRequest req, SolrQueryResponse rsp) { this.rsp = rsp; - this.allowPartialResults = - req != null ? req.getParams().getBool(CommonParams.PARTIAL_RESULTS, true) : true; + this.allowPartialResults = req == null || SolrQueryRequest.allowPartialResults(req.getParams()); if (req != null) { if (hasTimeLimit(req)) { limits.add(new TimeAllowedLimit(req)); @@ -106,14 +107,20 @@ public String formatExceptionMessage(String label) { * * @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. + * @throws QueryLimitsExceededException if {@link #allowPartialResults} is false and limits have + * been reached. */ public boolean maybeExitWithPartialResults(String label) throws QueryLimitsExceededException { if (isLimitsEnabled() && shouldExit()) { if (allowPartialResults) { if (rsp != null) { - rsp.setPartialResults(); + SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo(); + if (requestInfo == null) { + throw new SolrException( + SolrException.ErrorCode.SERVER_ERROR, + "No request active, but attempting to exit with partial results?"); + } + rsp.setPartialResults(requestInfo.getReq()); rsp.addPartialResponseDetail(formatExceptionMessage(label)); } return true; @@ -137,7 +144,7 @@ public boolean maybeExitWithPartialResults(String label) throws QueryLimitsExcee */ public String limitStatusMessage() { if (limits.isEmpty()) { - return "This request is unlimited."; + return UNLIMITED; } StringBuilder sb = new StringBuilder("Query limits: "); for (QueryTimeout limit : limits) { diff --git a/solr/core/src/java/org/apache/solr/search/QueryResult.java b/solr/core/src/java/org/apache/solr/search/QueryResult.java index 9c9a8e6f60d..900a72214e7 100755 --- a/solr/core/src/java/org/apache/solr/search/QueryResult.java +++ b/solr/core/src/java/org/apache/solr/search/QueryResult.java @@ -19,7 +19,8 @@ /** The result of a search. */ public class QueryResult { - private boolean partialResults; + // Object for back compatibility so that we render true not "true" in json + private Object partialResults; private Boolean segmentTerminatedEarly; private DocListAndSet docListAndSet; private CursorMark nextCursorMark; @@ -49,10 +50,15 @@ public void setDocSet(DocSet set) { } public boolean isPartialResults() { - return partialResults; + // omitted is equivalent to false/empty for java logic + return Boolean.parseBoolean(String.valueOf(partialResults)); } - public void setPartialResults(boolean partialResults) { + public boolean isPartialResultOmitted() { + return "omitted".equals(partialResults); + } + + public void setPartialResults(Object partialResults) { this.partialResults = partialResults; } 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 5a84339a775..ae9ee159791 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 @@ -17,6 +17,7 @@ package org.apache.solr.search.facet; import static org.apache.solr.common.util.Utils.fromJSONString; +import static org.apache.solr.response.SolrQueryResponse.haveCompleteResults; import java.io.IOException; import java.util.Collection; @@ -37,7 +38,6 @@ import org.apache.solr.handler.component.SearchComponent; 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.QueryContext; import org.noggit.CharArr; import org.noggit.JSONWriter; @@ -300,10 +300,8 @@ public void handleResponses(ResponseBuilder rb, ShardRequest sreq) { if (facet == null) { SimpleOrderedMap shardResponseHeader = (SimpleOrderedMap) rsp.getResponse().get("responseHeader"); - if (Boolean.TRUE.equals( - shardResponseHeader.getBooleanArg( - SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY))) { - rb.rsp.setPartialResults(); + if (!haveCompleteResults(shardResponseHeader)) { + rb.rsp.setPartialResults(rb.req); } continue; } diff --git a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java index bfebe8cf233..cd70c53ce33 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java @@ -16,6 +16,8 @@ */ package org.apache.solr.search.grouping; +import static org.apache.solr.response.SolrQueryResponse.partialResultsStatus; + import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; @@ -34,6 +36,8 @@ import org.apache.lucene.search.grouping.TermGroupSelector; import org.apache.lucene.search.grouping.ValueSourceGroupSelector; import org.apache.solr.common.util.NamedList; +import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.request.SolrRequestInfo; import org.apache.solr.schema.FieldType; import org.apache.solr.schema.SchemaField; import org.apache.solr.search.BitDocSet; @@ -122,7 +126,6 @@ public CommandHandler build() { private final boolean needDocSet; private final boolean truncateGroups; private final boolean includeHitCount; - private boolean partialResults = false; private int totalHitCount; private DocSet docSet; @@ -224,7 +227,15 @@ public NamedList> processResult( if (docSet != null) { queryResult.setDocSet(docSet); } - queryResult.setPartialResults(partialResults); + if (queryResult.isPartialResults()) { + queryResult.setPartialResults( + partialResultsStatus( + SolrRequestInfo.getRequest() + .map( + solrQueryRequest -> + SolrQueryRequest.disallowPartialResults(solrQueryRequest.getParams())) + .orElse(false))); + } return transformer.transform(commands); } @@ -256,7 +267,6 @@ private void searchWithTimeLimiter(Query query, ProcessedFilter filter, Collecto searcher.search(query, collector); } catch (TimeLimitingCollector.TimeExceededException | ExitableDirectoryReader.ExitingReaderException x) { - partialResults = true; log.warn("Query: {}; ", query, x); } 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 35fcf5e4b21..e9af407a3a8 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 @@ -33,6 +33,7 @@ import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; +import org.apache.solr.handler.component.HttpShardHandler; import org.apache.solr.handler.component.ResponseBuilder; import org.apache.solr.handler.component.ShardRequest; import org.apache.solr.handler.component.ShardResponse; @@ -104,8 +105,8 @@ public void process(ResponseBuilder rb, ShardRequest shardRequest) { } shardInfo.add(srsp.getShard(), nl); } - if (ShardParams.getShardsTolerantAsBool(rb.req.getParams()) && srsp.getException() != null) { - rb.rsp.setPartialResults(); + if (HttpShardHandler.getShardsTolerantAsBool(rb.req) && srsp.getException() != null) { + rb.rsp.setPartialResults(rb.req); 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 0a2eea9b9ee..206fd982964 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 @@ -34,6 +34,7 @@ import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; +import org.apache.solr.handler.component.HttpShardHandler; import org.apache.solr.handler.component.ResponseBuilder; import org.apache.solr.handler.component.ShardDoc; import org.apache.solr.handler.component.ShardRequest; @@ -116,8 +117,8 @@ public void process(ResponseBuilder rb, ShardRequest shardRequest) { } shardInfo.add(srsp.getShard(), individualShardInfo); } - if (ShardParams.getShardsTolerantAsBool(rb.req.getParams()) && srsp.getException() != null) { - rb.rsp.setPartialResults(); + if (HttpShardHandler.getShardsTolerantAsBool(rb.req) && srsp.getException() != null) { + rb.rsp.setPartialResults(rb.req); continue; // continue if there was an error and we're tolerant. } NamedList> secondPhaseResult = diff --git a/solr/core/src/java/org/apache/solr/search/stats/ExactStatsCache.java b/solr/core/src/java/org/apache/solr/search/stats/ExactStatsCache.java index 0d3f746acb0..59f0eb4d8e3 100644 --- a/solr/core/src/java/org/apache/solr/search/stats/ExactStatsCache.java +++ b/solr/core/src/java/org/apache/solr/search/stats/ExactStatsCache.java @@ -41,6 +41,7 @@ import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.util.NamedList; +import org.apache.solr.handler.component.HttpShardHandler; import org.apache.solr.handler.component.ResponseBuilder; import org.apache.solr.handler.component.ShardRequest; import org.apache.solr.handler.component.ShardResponse; @@ -100,8 +101,7 @@ protected ShardRequest doRetrieveStatsRequest(ResponseBuilder rb) { protected void doMergeToGlobalStats(SolrQueryRequest req, List responses) { Set allTerms = new HashSet<>(); for (ShardResponse r : responses) { - if ("true".equalsIgnoreCase(req.getParams().get(ShardParams.SHARDS_TOLERANT)) - && r.getException() != null) { + if (HttpShardHandler.getShardsTolerantAsBool(req) && r.getException() != null) { // Can't expect stats if there was an exception for this request on any shard // this should only happen when using shards.tolerant=true log.debug("Exception shard response={}", r); diff --git a/solr/core/src/java/org/apache/solr/util/SolrResponseUtil.java b/solr/core/src/java/org/apache/solr/util/SolrResponseUtil.java index eb12dc9d779..660465552d0 100644 --- a/solr/core/src/java/org/apache/solr/util/SolrResponseUtil.java +++ b/solr/core/src/java/org/apache/solr/util/SolrResponseUtil.java @@ -20,8 +20,8 @@ import java.util.Objects; import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.common.SolrException; -import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.util.NamedList; +import org.apache.solr.handler.component.HttpShardHandler; import org.apache.solr.handler.component.ResponseBuilder; import org.apache.solr.handler.component.ShardResponse; import org.apache.solr.response.SolrQueryResponse; @@ -64,7 +64,7 @@ public static Object getSubsectionFromShardResponse( } } } catch (Exception ex) { - if (rb != null && ShardParams.getShardsTolerantAsBool(rb.req.getParams())) { + if (rb != null && HttpShardHandler.getShardsTolerantAsBool(rb.req)) { return null; } else { throw new SolrException( diff --git a/solr/core/src/test/org/apache/solr/search/TestCpuAllowedLimit.java b/solr/core/src/test/org/apache/solr/search/TestCpuAllowedLimit.java index 2a61aa9eebd..274994099f1 100644 --- a/solr/core/src/test/org/apache/solr/search/TestCpuAllowedLimit.java +++ b/solr/core/src/test/org/apache/solr/search/TestCpuAllowedLimit.java @@ -166,7 +166,7 @@ public void testDistribLimit() throws Exception { "timeAllowed", "500")); // System.err.println("rsp=" + rsp.jsonStr()); - assertNotNull("should have partial results", rsp.getHeader().get("partialResults")); + assertEquals("should have partial results", true, rsp.getHeader().get("partialResults")); log.info("--- timeAllowed, partial results, multithreading ---"); rsp = @@ -222,6 +222,28 @@ public void testDistribLimit() throws Exception { "false")); // System.err.println("rsp=" + rsp.jsonStr()); assertNotNull("should have partial results", rsp.getHeader().get("partialResults")); + log.info("--- cpuAllowed 1, partial results omitted ---"); + rsp = + solrClient.query( + COLLECTION, + params( + "q", + "id:*", + "sort", + "id desc", + "stages", + "prepare,process", + "cpuAllowed", + "100", + "partialResults", + "false", + "multiThreaded", + "false", + "_", + "foo")); + String s = rsp.jsonStr(); + System.err.println("rsp=" + s); + assertEquals("should have partial results", "omitted", rsp.getHeader().get("partialResults")); // cpuAllowed set, should return partial results log.info("--- cpuAllowed 2, partial results, multi-threaded ---"); 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 80e92f48289..41863284b99 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 @@ -324,9 +324,20 @@ The default value of this parameter is blank, which causes no extra "explain inf 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 `true` (default) then Solr will return the results collected prior to detecting the limit violation. +If `shards.tolerant=true` is also set, all non-limited responses will still be collected. If you are seeking a best-effort response use of the xref:deployment-guide:solrcloud-distributed-requests.adoc#shards-tolerant-parameter[shards.tolerant Parameter] is recommended. + +If incomplete results are returned the `partialResults` response header will be set to `true` + +WARNING: 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 upon reaching a limit, Solr will stop collecting results and will not return any partial results already collected. +In this case, the partialResults response header will be set to `omitted` and no result documents are returned. + +NOTE: The response will be 200 OK because the system has correctly provided the requested behavior. +This is not an error condition. -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 diff --git a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc index fed3fc7fd1b..399ceba5926 100644 --- a/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc +++ b/solr/solr-ref-guide/modules/upgrade-notes/pages/major-changes-in-solr-9.adoc @@ -71,6 +71,12 @@ Due to changes in Lucene 9, that isn't possible any more. === Configuration In solrconfig.xml, the `numVersionBuckets` and `versionBucketLockTimeoutMs` settings are now obsolete and ignored; a warning will be logged if specified. +=== Partial Results +When query limits are in use and partial results are not desirable (i.e. reporting or quantitative usages of search) +users may pass `partialResults=false`. +This feature has been improved to eliminate some wasted processing. +In the presence of this parameter, queries encountering limits will return zero results, and to make it clear when this has happened, the `partialResults` response header will be set to `"omitted"` + == Solr 9.7 === SchemaVersion upgraded to 1.7 The default schemaVersion is now 1.7. diff --git a/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java b/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java index 273db7651e8..a96a5d4943d 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java @@ -16,8 +16,6 @@ */ package org.apache.solr.common.params; -import org.apache.solr.common.util.StrUtils; - /** * Parameters used for distributed search. * @@ -59,7 +57,7 @@ public interface ShardParams { /** Request detailed match info for each shard (true/false) */ String SHARDS_INFO = "shards.info"; - /** Should things fail if there is an error? (true/false/{@value #REQUIRE_ZK_CONNECTED}) */ + /** Should things fail if there is an error? (true/false/requireZkConnected) */ String SHARDS_TOLERANT = "shards.tolerant"; /** query purpose for shard requests */ @@ -104,26 +102,10 @@ public interface ShardParams { String DISTRIB_SINGLE_PASS = "distrib.singlePass"; /** - * Throw an error from search requests when the {@value #SHARDS_TOLERANT} param has this value and - * ZooKeeper is not connected. + * Throw an error from search requests when the {@value ShardParams#SHARDS_TOLERANT} param has + * this value and ZooKeeper is not connected. * - * @see #getShardsTolerantAsBool(SolrParams) + *

See also HttpShardHandler's getShardsTolerantAsBool(SolrQueryRequest) */ String REQUIRE_ZK_CONNECTED = "requireZkConnected"; - - /** - * Parse the {@value #SHARDS_TOLERANT} param from params as a boolean; accepts - * {@value #REQUIRE_ZK_CONNECTED} as a valid value indicating false. - * - *

By default, returns false when {@value #SHARDS_TOLERANT} is not set in - * params. - */ - static boolean getShardsTolerantAsBool(SolrParams params) { - String shardsTolerantValue = params.get(SHARDS_TOLERANT); - if (null == shardsTolerantValue || shardsTolerantValue.equals(REQUIRE_ZK_CONNECTED)) { - return false; - } else { - return StrUtils.parseBool(shardsTolerantValue.trim()); // throw an exception if non-boolean - } - } } diff --git a/solr/solrj/src/test/org/apache/solr/common/params/ShardParamsTest.java b/solr/solrj/src/test/org/apache/solr/common/params/ShardParamsTest.java index 63f09d644d3..32ebca5c75e 100644 --- a/solr/solrj/src/test/org/apache/solr/common/params/ShardParamsTest.java +++ b/solr/solrj/src/test/org/apache/solr/common/params/ShardParamsTest.java @@ -18,6 +18,9 @@ import org.apache.solr.SolrTestCase; import org.apache.solr.common.SolrException; +import org.apache.solr.handler.component.HttpShardHandler; +import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.request.SolrQueryRequestBase; import org.junit.Test; /** @@ -76,29 +79,31 @@ public void testDistribSinglePass() { @Test public void testGetShardsTolerantAsBool() { ModifiableSolrParams params = new ModifiableSolrParams(); + SolrQueryRequest paramsSupplier = new SolrQueryRequestBase(null, params) {}; // shards.tolerant param is not set; default should be false - assertFalse(ShardParams.getShardsTolerantAsBool(params)); + assertFalse(HttpShardHandler.getShardsTolerantAsBool(paramsSupplier)); // shards.tolerant boolean true param should return true for (String trueValue : new String[] {"true", "yes", "on"}) { params.set(ShardParams.SHARDS_TOLERANT, trueValue); - assertTrue(ShardParams.getShardsTolerantAsBool(params)); + assertTrue(HttpShardHandler.getShardsTolerantAsBool(paramsSupplier)); } // shards.tolerant boolean false param should return false for (String falseValue : new String[] {"false", "no", "off"}) { params.set(ShardParams.SHARDS_TOLERANT, falseValue); - assertFalse(ShardParams.getShardsTolerantAsBool(params)); + assertFalse(HttpShardHandler.getShardsTolerantAsBool(paramsSupplier)); } // shards.tolerant=requireZkConnected should return false params.set(ShardParams.SHARDS_TOLERANT, ShardParams.REQUIRE_ZK_CONNECTED); - assertFalse(ShardParams.getShardsTolerantAsBool(params)); + assertFalse(HttpShardHandler.getShardsTolerantAsBool(paramsSupplier)); // values that aren't "requireZkConnected" or boolean should throw an exception params.set(ShardParams.SHARDS_TOLERANT, "bogusValue"); Exception exception = - expectThrows(SolrException.class, () -> ShardParams.getShardsTolerantAsBool(params)); + expectThrows( + SolrException.class, () -> HttpShardHandler.getShardsTolerantAsBool(paramsSupplier)); assertTrue( exception.getMessage(), exception.getMessage().startsWith("invalid boolean value: ")); }