SOLR-14384: SolrRequestInfo now stacks internally.

* "set" now MUST pair with a "clear"
* fixes SolrIndexSearcher.warm which should have re-instated previous SRI
* cleans up some SRI set/clear users

Closes #1527
This commit is contained in:
Nazerke Seidan 2020-06-16 01:46:50 -04:00 committed by David Smiley
parent 47cffbcdd8
commit 2da71c2a40
No known key found for this signature in database
GPG Key ID: 6FDFF3BF6796FD4A
10 changed files with 102 additions and 107 deletions

View File

@ -143,6 +143,10 @@ Improvements
* SOLR-14442: bin/solr and bin\solr.cmd invoke jstack <SOLR_PID> before forceful termination, if jstack is available.
Also, bin\solr.cmd executes forceful termination even port is unbinded already (Christine Poerschke via Mikhail Khludnev).
* SOLR-14384: SolrRequestInfo now stacks internally when a new request is set/clear'ed.
Also fixes SolrIndexSearcher.warm which should have re-instated previous SRI.
(Nazerke Seidan, David Smiley)
Optimizations
---------------------
* SOLR-8306: Do not collect expand documents when expand.rows=0 (Marshall Sanders, Amelia Henderson)

View File

@ -271,8 +271,10 @@ public class EmbeddedSolrServer extends SolrClient {
} catch (Exception ex) {
throw new SolrServerException(ex);
} finally {
if (req != null) req.close();
SolrRequestInfo.clearRequestInfo();
if (req != null) {
req.close();
SolrRequestInfo.clearRequestInfo();
}
}
}

View File

@ -51,9 +51,7 @@ public class QuerySenderListener extends AbstractSolrEventListener {
log.debug("QuerySenderListener sending requests to {}", newSearcher);
List<NamedList> allLists = (List<NamedList>)getArgs().get("queries");
if (allLists == null) return;
boolean createNewReqInfo = SolrRequestInfo.getRequestInfo() == null;
for (NamedList nlst : allLists) {
SolrQueryRequest req = null;
try {
// bind the request to a particular searcher (the newSearcher)
NamedList params = addEventParms(currentSearcher, nlst);
@ -61,42 +59,41 @@ public class QuerySenderListener extends AbstractSolrEventListener {
if (params.get(DISTRIB) == null) {
params.add(DISTRIB, false);
}
req = new LocalSolrQueryRequest(getCore(),params) {
SolrQueryRequest req = new LocalSolrQueryRequest(getCore(),params) {
@Override public SolrIndexSearcher getSearcher() { return searcher; }
@Override public void close() { }
};
SolrQueryResponse rsp = new SolrQueryResponse();
if (createNewReqInfo) {
// SolrRequerstInfo for this thread could have been transferred from the parent
// thread.
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
}
getCore().execute(getCore().getRequestHandler(req.getParams().get(CommonParams.QT)), req, rsp);
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
try {
getCore().execute(getCore().getRequestHandler(req.getParams().get(CommonParams.QT)), req, rsp);
// Retrieve the Document instances (not just the ids) to warm
// the OS disk cache, and any Solr document cache. Only the top
// level values in the NamedList are checked for DocLists.
NamedList values = rsp.getValues();
for (int i=0; i<values.size(); i++) {
Object o = values.getVal(i);
if (o instanceof ResultContext) {
o = ((ResultContext)o).getDocList();
}
if (o instanceof DocList) {
DocList docs = (DocList)o;
for (DocIterator iter = docs.iterator(); iter.hasNext();) {
newSearcher.doc(iter.nextDoc());
// Retrieve the Document instances (not just the ids) to warm
// the OS disk cache, and any Solr document cache. Only the top
// level values in the NamedList are checked for DocLists.
NamedList values = rsp.getValues();
for (int i=0; i<values.size(); i++) {
Object o = values.getVal(i);
if (o instanceof ResultContext) {
o = ((ResultContext)o).getDocList();
}
if (o instanceof DocList) {
DocList docs = (DocList)o;
for (DocIterator iter = docs.iterator(); iter.hasNext();) {
newSearcher.doc(iter.nextDoc());
}
}
}
} finally {
try {
req.close();
} finally {
SolrRequestInfo.clearRequestInfo();
}
}
} catch (Exception e) {
// do nothing... we want to continue with the other requests.
// the failure should have already been logged.
} finally {
if (req != null) req.close();
if (createNewReqInfo) SolrRequestInfo.clearRequestInfo();
}
}
log.info("QuerySenderListener done.");

View File

@ -58,19 +58,18 @@ import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.core.CloseHook;
import org.apache.solr.core.PluginBag;
import org.apache.solr.core.SolrCore;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.update.SolrCoreState;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.VersionInfo;
import org.apache.solr.update.processor.DistributedUpdateProcessor;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.plugin.SolrCoreAware;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -776,12 +775,6 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
success = replicationHandler.doFetch(solrParams, false).getSuccessful();
// this is required because this callable can race with HttpSolrCall#destroy
// which clears the request info.
// Applying buffered updates fails without the following line because LogReplayer
// also tries to set request info and fails with AssertionError
SolrRequestInfo.clearRequestInfo();
Future<UpdateLog.RecoveryInfo> future = ulog.applyBufferedUpdates();
if (future == null) {
// no replay needed

View File

@ -21,6 +21,7 @@ import java.io.Closeable;
import java.lang.invoke.MethodHandles;
import java.security.Principal;
import java.util.Date;
import java.util.Deque;
import java.util.LinkedList;
import java.util.List;
import java.util.TimeZone;
@ -36,9 +37,12 @@ import org.apache.solr.util.TimeZoneUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Information about the Solr request/response held in a {@link ThreadLocal}. */
public class SolrRequestInfo {
protected final static ThreadLocal<SolrRequestInfo> threadLocal = new ThreadLocal<>();
protected static final int MAX_STACK_SIZE = 10;
protected static final ThreadLocal<Deque<SolrRequestInfo>> threadLocal = ThreadLocal.withInitial(LinkedList::new);
protected SolrQueryRequest req;
protected SolrQueryResponse rsp;
@ -52,35 +56,62 @@ public class SolrRequestInfo {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static SolrRequestInfo getRequestInfo() {
return threadLocal.get();
Deque<SolrRequestInfo> stack = threadLocal.get();
if (stack.isEmpty()) return null;
return stack.peek();
}
/**
* Adds the SolrRequestInfo onto a stack held in a {@link ThreadLocal}.
* Remember to call {@link #clearRequestInfo()}!
*/
public static void setRequestInfo(SolrRequestInfo info) {
// TODO: temporary sanity check... this can be changed to just an assert in the future
SolrRequestInfo prev = threadLocal.get();
if (prev != null) {
log.error("Previous SolrRequestInfo was not closed! req={}", prev.req.getOriginalParams());
log.error("prev == info : {}", prev.req == info.req, new RuntimeException());
Deque<SolrRequestInfo> stack = threadLocal.get();
if (info == null) {
throw new IllegalArgumentException("SolrRequestInfo is null");
} else if (stack.size() <= MAX_STACK_SIZE) {
stack.push(info);
} else {
assert false : "SolrRequestInfo Stack is full";
log.error("SolrRequestInfo Stack is full");
}
assert prev == null;
threadLocal.set(info);
}
/** Removes the most recent SolrRequestInfo from the stack */
public static void clearRequestInfo() {
try {
SolrRequestInfo info = threadLocal.get();
if (info != null && info.closeHooks != null) {
for (Closeable hook : info.closeHooks) {
try {
hook.close();
} catch (Exception e) {
SolrException.log(log, "Exception during close hook", e);
}
Deque<SolrRequestInfo> stack = threadLocal.get();
if (stack.isEmpty()) {
assert false : "clearRequestInfo called too many times";
log.error("clearRequestInfo called too many times");
} else {
SolrRequestInfo info = stack.pop();
closeHooks(info);
}
}
/**
* This reset method is more of a protection mechanism as
* we expect it to be empty by now because all "set" calls need to be balanced with a "clear".
*/
public static void reset() {
Deque<SolrRequestInfo> stack = threadLocal.get();
boolean isEmpty = stack.isEmpty();
while (!stack.isEmpty()) {
SolrRequestInfo info = stack.pop();
closeHooks(info);
}
assert isEmpty : "SolrRequestInfo Stack should have been cleared.";
}
private static void closeHooks(SolrRequestInfo info) {
if (info.closeHooks != null) {
for (Closeable hook : info.closeHooks) {
try {
hook.close();
} catch (Exception e) {
SolrException.log(log, "Exception during close hook", e);
}
}
} finally {
threadLocal.remove();
}
}
@ -183,14 +214,16 @@ public class SolrRequestInfo {
public void set(@SuppressWarnings({"rawtypes"})AtomicReference ctx) {
SolrRequestInfo me = (SolrRequestInfo) ctx.get();
if (me != null) {
ctx.set(null);
SolrRequestInfo.setRequestInfo(me);
}
}
@Override
public void clean(@SuppressWarnings({"rawtypes"})AtomicReference ctx) {
SolrRequestInfo.clearRequestInfo();
if (ctx.get() != null) {
SolrRequestInfo.clearRequestInfo();
}
SolrRequestInfo.reset();
}
};
}

View File

@ -20,14 +20,12 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.Callable;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TotalHits;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
@ -36,9 +34,7 @@ import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.response.ResultContext;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.search.DocList;
import org.apache.solr.search.DocSlice;
import org.apache.solr.search.JoinQParserPlugin;
@ -329,51 +325,14 @@ class SubQueryAugmenter extends DocTransformer {
final SolrParams docWithDeprefixed = SolrParams.wrapDefaults(
new DocRowParams(doc, prefix, separator), baseSubParams);
try {
Callable<QueryResponse> subQuery = new Callable<QueryResponse>() {
@Override
public QueryResponse call() throws Exception {
try {
return new QueryResponse(
server.request(
new QueryRequest(docWithDeprefixed), coreName)
, server);
} finally {
}
}
};
QueryResponse response =
SolrRequestInfoSuspender.doInSuspension(subQuery);
final SolrDocumentList docList = response.getResults();
QueryResponse rsp = server.query(coreName, docWithDeprefixed);
SolrDocumentList docList = rsp.getResults();
doc.setField(getName(), new Result(docList));
} catch (Exception e) {
String docString = doc.toString();
throw new SolrException(ErrorCode.BAD_REQUEST, "while invoking " +
name + ":[subquery"+ (coreName!=null ? "fromIndex="+coreName : "") +"] on doc=" +
docString.substring(0, Math.min(100, docString.length())), e.getCause());
} finally {}
}
// look ma!! no hands..
final static class SolrRequestInfoSuspender extends SolrRequestInfo {
private SolrRequestInfoSuspender(SolrQueryRequest req, SolrQueryResponse rsp) {
super(req, rsp);
}
/** Suspends current SolrRequestInfo invoke the given action, and resumes then */
static <T> T doInSuspension(Callable<T> action) throws Exception {
final SolrRequestInfo info = threadLocal.get();
try {
threadLocal.remove();
return action.call();
} finally {
setRequestInfo(info);
}
}
}
}

View File

@ -2157,7 +2157,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
};
final SolrQueryResponse rsp = new SolrQueryResponse();
SolrRequestInfo.clearRequestInfo();
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
try {
cacheList[i].warm(this, old.cacheList[i]);

View File

@ -135,8 +135,8 @@ public class DirectSolrConnection
} finally {
if (req != null) {
req.close();
SolrRequestInfo.clearRequestInfo();
}
SolrRequestInfo.clearRequestInfo();
}
}

View File

@ -163,6 +163,7 @@ public class HttpSolrCall {
protected final boolean retry;
protected SolrCore core = null;
protected SolrQueryRequest solrReq = null;
private boolean mustClearSolrRequestInfo = false;
protected SolrRequestHandler handler = null;
protected final SolrParams queryParams;
protected String path;
@ -413,6 +414,7 @@ public class HttpSolrCall {
if (path.equals("/schema") || path.startsWith("/schema/")) {
solrReq = parser.parse(core, path, req);
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(solrReq, new SolrQueryResponse()));
mustClearSolrRequestInfo = true;
if (path.equals(req.getServletPath())) {
// avoid endless loop - pass through to Restlet via webapp
action = PASSTHROUGH;
@ -564,6 +566,7 @@ public class HttpSolrCall {
return RETURN;
case REMOTEQUERY:
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, new SolrQueryResponse(), action));
mustClearSolrRequestInfo = true;
remoteQuery(coreUrl + path, resp);
return RETURN;
case PROCESS:
@ -580,6 +583,7 @@ public class HttpSolrCall {
* Content-Type)
*/
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(solrReq, solrRsp, action));
mustClearSolrRequestInfo = true;
execute(solrRsp);
if (shouldAudit()) {
EventType eventType = solrRsp.getException() == null ? EventType.COMPLETED : EventType.ERROR;
@ -652,7 +656,9 @@ public class HttpSolrCall {
try {
if (core != null) core.close();
} finally {
SolrRequestInfo.clearRequestInfo();
if (mustClearSolrRequestInfo) {
SolrRequestInfo.clearRequestInfo();
}
}
AuthenticationPlugin authcPlugin = cores.getAuthenticationPlugin();
if (authcPlugin != null) authcPlugin.closeRequest();

View File

@ -78,6 +78,7 @@ import org.apache.solr.metrics.MetricsMap;
import org.apache.solr.metrics.OperatingSystemMetricSet;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.metrics.SolrMetricProducer;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.security.AuditEvent;
import org.apache.solr.security.AuthenticationPlugin;
import org.apache.solr.security.PKIAuthenticationPlugin;
@ -439,6 +440,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
GlobalTracer.get().clearContext();
consumeInputFully(request, response);
SolrRequestInfo.reset();
SolrRequestParsers.cleanupMultipartFiles(request);
}
}