Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,6 @@
import org.apache.solr.search.QueryUtils;
import org.apache.solr.search.ReturnFields;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SolrQueryTimeoutImpl;
import org.apache.solr.search.SolrReturnFields;
import org.apache.solr.search.SortSpec;
import org.apache.solr.search.SyntaxError;
Expand Down Expand Up @@ -97,7 +96,6 @@ public class MoreLikeThisHandler extends RequestHandlerBase {
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
SolrParams params = req.getParams();

SolrQueryTimeoutImpl.set(req);
try {

// Set field flags
Expand Down Expand Up @@ -267,8 +265,6 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
}
} catch (ExitableDirectoryReader.ExitingReaderException ex) {
log.warn("Query: {}; ", req.getParamString(), ex);
} finally {
SolrQueryTimeoutImpl.reset();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.CursorMark;
import org.apache.solr.search.SolrQueryTimeoutImpl;
import org.apache.solr.search.SortSpec;
import org.apache.solr.search.facet.FacetModule;
import org.apache.solr.security.AuthorizationContext;
Expand Down Expand Up @@ -457,7 +456,6 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
if (!rb.isDistrib) {
// a normal non-distributed request

SolrQueryTimeoutImpl.set(req);
try {
// The semantics of debugging vs not debugging are different enough that
// it makes sense to have two control loops
Expand Down Expand Up @@ -502,8 +500,6 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
.getResponseHeader()
.asShallowMap()
.put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
} finally {
SolrQueryTimeoutImpl.reset();
}
} else {
// a distributed request
Expand Down
34 changes: 34 additions & 0 deletions solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.handler.component.ResponseBuilder;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.QueryLimits;
import org.apache.solr.servlet.SolrDispatchFilter;
import org.apache.solr.util.TimeZoneUtils;
import org.slf4j.Logger;
Expand All @@ -43,6 +44,7 @@ public class SolrRequestInfo {

private static final ThreadLocal<Deque<SolrRequestInfo>> threadLocal =
ThreadLocal.withInitial(ArrayDeque::new);
static final Object LIMITS_KEY = new Object();

private int refCount = 1; // prevent closing when still used

Expand Down Expand Up @@ -75,7 +77,13 @@ public static void setRequestInfo(SolrRequestInfo info) {
} else if (stack.size() > MAX_STACK_SIZE) {
assert false : "SolrRequestInfo Stack is full";
log.error("SolrRequestInfo Stack is full");
} else if (!stack.isEmpty() && info.req != null) {
// New SRI instances inherit limits from prior SRI regardless of parameters.
// This ensures limits cannot be changed or removed for a given thread once set.
// if req is null limits will be an empty instance with no limits anyway.
info.req.getContext().put(LIMITS_KEY, stack.peek().getLimits());
}
info.initQueryLimits();
log.trace("{} {}", info, "setRequestInfo()");
assert !info.isClosed() : "SRI is already closed (odd).";
stack.push(info);
Expand Down Expand Up @@ -210,6 +218,32 @@ public void addCloseHook(Closeable hook) {
}
}

/**
* This call creates the QueryLimits object and any required implementations of {@link
* org.apache.lucene.index.QueryTimeout}. Any code before this call will not be subject to the
* limitations set on the request. Note that calling {@link #getLimits()} has the same effect as
* this method.
*
* @see #getLimits()
*/
private void initQueryLimits() {
// This method only exists for code clarity reasons.
getLimits();
}

/**
* Get the query limits for the current request. This will trigger the creation of the (possibly
* empty) {@link QueryLimits} object if it has not been created, and will then return the same
* object on every subsequent invocation.
*
* @return The {@code QueryLimits} object for the current requet.
*/
public QueryLimits getLimits() {
return req == null
? QueryLimits.NONE
: (QueryLimits) req.getContext().computeIfAbsent(LIMITS_KEY, (k) -> new QueryLimits(req));
}

public SolrDispatchFilter.Action getAction() {
return action;
}
Expand Down
95 changes: 95 additions & 0 deletions solr/core/src/java/org/apache/solr/search/QueryLimits.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* 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 static org.apache.solr.search.SolrQueryTimeLimit.hasTimeLimit;

import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.QueryTimeout;
import org.apache.solr.request.SolrQueryRequest;

/**
* Represents the limitations on the query. These limits might be wall clock time, cpu time, memory,
* or other resource limits. Exceeding any specified limit will cause {@link #shouldExit()} to
* return true the next time it is checked (it may be checked in either Lucene code or Solr code)
*/
public class QueryLimits implements QueryTimeout {
private final List<QueryTimeout> limits =
new ArrayList<>(3); // timeAllowed, cpu, and memory anticipated

public static QueryLimits NONE = new QueryLimits();

private QueryLimits() {}

/**
* 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.
*/
public QueryLimits(SolrQueryRequest req) {
if (hasTimeLimit(req)) {
limits.add(new SolrQueryTimeLimit(req));
}
}

@Override
public boolean shouldExit() {
for (QueryTimeout limit : limits) {
if (limit.shouldExit()) {
return true;
}
}
return false;
}

/**
* Method to diagnose limit exceeded. Note that while this should always list the exceeded limit,
* it may also nominate additional limits that have been exceeded since the actual check that
* cause the failure. This gap is intentional to avoid overly complicated (and possibly expensive)
* tracking code that would have to run within the shouldExit method. This method should only be
* used to report a failure since it incurs the cost of rechecking every configured limit and does
* not short circuit.
*
* @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;
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();
}
}

public boolean isTimeoutEnabled() {
return !limits.isEmpty();
}
}
23 changes: 15 additions & 8 deletions solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
Original file line number Diff line number Diff line change
Expand Up @@ -204,7 +204,10 @@ private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader)
assert reader != null;
reader = UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMapper());
if (useExitableDirectoryReader) { // SOLR-16693 legacy; may be removed. Probably inefficient.
reader = ExitableDirectoryReader.wrap(reader, SolrQueryTimeoutImpl.getInstance());
SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
assert requestInfo != null;
QueryLimits limits = requestInfo.getLimits();
reader = ExitableDirectoryReader.wrap(reader, limits);
}
return reader;
}
Expand Down Expand Up @@ -714,8 +717,10 @@ public QueryResult search(QueryResult qr, QueryCommand cmd) throws IOException {
@Override
protected void search(List<LeafReaderContext> leaves, Weight weight, Collector collector)
throws IOException {
final var queryTimeout = SolrQueryTimeoutImpl.getInstance();
if (useExitableDirectoryReader || queryTimeout.isTimeoutEnabled() == false) {
SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
if (useExitableDirectoryReader
|| requestInfo == null
|| !requestInfo.getLimits().isTimeoutEnabled()) {
// no timeout. Pass through to super class
super.search(leaves, weight, collector);
} else {
Expand All @@ -725,10 +730,11 @@ protected void search(List<LeafReaderContext> leaves, Weight weight, Collector c
// So we need to make a new IndexSearcher instead of using "this".
new IndexSearcher(reader) { // cheap, actually!
void searchWithTimeout() throws IOException {
setTimeout(queryTimeout.makeLocalImpl());
setTimeout(requestInfo.getLimits()); // Lucene's method name is less than ideal here...
super.search(leaves, weight, collector); // FYI protected access
if (timedOut()) {
throw new TimeAllowedExceededFromScorerException("timeAllowed exceeded");
throw new LimitExceededFromScorerException(
"Limits exceeded! " + requestInfo.getLimits().limitStatusMessage());
}
}
}.searchWithTimeout();
Expand All @@ -740,10 +746,10 @@ void searchWithTimeout() throws IOException {
* Further, from the low level Lucene {@code org.apache.lucene.search.TimeLimitingBulkScorer}.
* Extending {@code ExitableDirectoryReader.ExitingReaderException} is for legacy reasons.
*/
public static class TimeAllowedExceededFromScorerException
public static class LimitExceededFromScorerException
extends ExitableDirectoryReader.ExitingReaderException {

public TimeAllowedExceededFromScorerException(String msg) {
public LimitExceededFromScorerException(String msg) {
super(msg);
}
}
Expand Down Expand Up @@ -972,7 +978,8 @@ private DocSet getAndCacheDocSet(Query query) throws IOException {
}

DocSet answer;
if (SolrQueryTimeoutImpl.getInstance().isTimeoutEnabled()) {
SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
if (requestInfo != null && requestInfo.getLimits().isTimeoutEnabled()) {
// If there is a possibility of timeout for this query, then don't reserve a computation slot.
// Further, we can't naively wait for an in progress computation to finish, because if we time
// out before it does then we won't even have partial results to provide. We could possibly
Expand Down
66 changes: 66 additions & 0 deletions solr/core/src/java/org/apache/solr/search/SolrQueryTimeLimit.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
/*
* 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 static java.lang.System.nanoTime;

import java.util.concurrent.TimeUnit;
import org.apache.lucene.index.QueryTimeout;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.request.SolrQueryRequest;

/**
* Enforces a wall clock based timeout on a given SolrQueryRequest. This class holds the logic for
* the {@code timeAllowed} query parameter. Note that timeAllowed will be ignored for
* <strong><em>local</em></strong> processing of sub-queries in cases where the parent query already
* has {@code timeAllowed} set. Essentially only one timeAllowed can be specified for any thread
* executing a query. This is to ensure that subqueies don't escape from the intended limit
*/
public class SolrQueryTimeLimit implements QueryTimeout {

private final long timeoutAt;

/**
* Create an object to represent a time limit for the current request.
*
* @param req A solr request that has a value for {@code timeAllowed}
* @throws IllegalArgumentException if the request does not contain timeAllowed parameter. This
* should be validated with {@link #hasTimeLimit(SolrQueryRequest)} prior to constructing this
* object
*/
public SolrQueryTimeLimit(SolrQueryRequest req) {
// reduce by time already spent
long reqTimeAllowed = req.getParams().getLong(CommonParams.TIME_ALLOWED, -1L);

if (reqTimeAllowed == -1L) {
throw new IllegalArgumentException(
"Check for limit with hasTimeLimit(req) before creating a SolrQueryTimeLimit");
}
long timeAllowed = reqTimeAllowed - (long) req.getRequestTimer().getTime();
long nanosAllowed = TimeUnit.NANOSECONDS.convert(timeAllowed, TimeUnit.MILLISECONDS);
timeoutAt = nanoTime() + nanosAllowed;
}

static boolean hasTimeLimit(SolrQueryRequest req) {
return req.getParams().getLong(CommonParams.TIME_ALLOWED, -1L) >= 0L;
}

@Override
public boolean shouldExit() {
return timeoutAt - nanoTime() < 0L;
}
}
Loading