mirror of https://github.com/apache/lucene.git
SOLR-6275: Improve accuracy of QTime reporting
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1663829 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
8d6657a246
commit
b29aef7954
|
@ -237,6 +237,8 @@ Other Changes
|
|||
|
||||
* SOLR-7151: SolrClient query methods throw IOException (Alan Woodward)
|
||||
|
||||
* SOLR-6275: Improve accuracy of QTime reporting (Ramkumar Aiyengar)
|
||||
|
||||
================== 5.0.0 ==================
|
||||
|
||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||
|
|
|
@ -2031,9 +2031,9 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
requestLog.info(rsp.getToLogAsString(logid));
|
||||
}
|
||||
|
||||
if (log.isWarnEnabled()) {
|
||||
final int qtime = (int)(rsp.getEndTime() - req.getStartTime());
|
||||
if (slowQueryThresholdMillis >= 0 && qtime >= slowQueryThresholdMillis) {
|
||||
if (log.isWarnEnabled() && slowQueryThresholdMillis >= 0) {
|
||||
final long qtime = (long) (req.getRequestTimer().getTime());
|
||||
if (qtime >= slowQueryThresholdMillis) {
|
||||
log.warn("slow: " + rsp.getToLogAsString(logid));
|
||||
}
|
||||
}
|
||||
|
@ -2067,7 +2067,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
(SolrRequestHandler handler, SolrQueryRequest req, SolrQueryResponse rsp) {
|
||||
// TODO should check that responseHeader has not been replaced by handler
|
||||
NamedList<Object> responseHeader = rsp.getResponseHeader();
|
||||
final int qtime=(int)(rsp.getEndTime() - req.getStartTime());
|
||||
final int qtime=(int)(req.getRequestTimer().getTime());
|
||||
int status = 0;
|
||||
Exception exception = rsp.getException();
|
||||
if( exception != null ){
|
||||
|
|
|
@ -169,7 +169,6 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
|
|||
return components;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception
|
||||
{
|
||||
|
@ -190,8 +189,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
|
|||
SolrPluginUtils.getDebugInterests(req.getParams().getParams(CommonParams.DEBUG), rb);
|
||||
}
|
||||
|
||||
final RTimer timer = rb.isDebug() ? new RTimer() : null;
|
||||
|
||||
final RTimer timer = rb.isDebug() ? req.getRequestTimer() : null;
|
||||
|
||||
ShardHandler shardHandler1 = shardHandlerFactory.getShardHandler();
|
||||
shardHandler1.checkDistributed(rb);
|
||||
|
@ -237,7 +235,6 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
|
|||
rb.getTimer().stop();
|
||||
}
|
||||
subt.stop();
|
||||
timer.stop();
|
||||
|
||||
// add the timing info
|
||||
if (rb.isDebugTimings()) {
|
||||
|
@ -395,7 +392,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
|
|||
nl.add("maxScore", rb.getResults().docList.maxScore());
|
||||
}
|
||||
nl.add("shardAddress", rb.shortCircuitedURL);
|
||||
nl.add("time", rsp.getEndTime()-req.getStartTime()); // elapsed time of this request so far
|
||||
nl.add("time", req.getRequestTimer().getTime()); // elapsed time of this request so far
|
||||
|
||||
int pos = rb.shortCircuitedURL.indexOf("://");
|
||||
String shardInfoName = pos != -1 ? rb.shortCircuitedURL.substring(pos+3) : rb.shortCircuitedURL;
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.solr.request;
|
|||
import org.apache.solr.response.SolrQueryResponse;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.schema.IndexSchema;
|
||||
import org.apache.solr.util.RTimer;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ContentStream;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
|
@ -63,9 +64,15 @@ public interface SolrQueryRequest {
|
|||
*/
|
||||
public void close();
|
||||
|
||||
/** The start time of this request in milliseconds */
|
||||
/** The start time of this request in milliseconds.
|
||||
* Use this only if you need the absolute system time at the start of the request,
|
||||
* getRequestTimer() provides a more accurate mechanism for timing purposes.
|
||||
*/
|
||||
public long getStartTime();
|
||||
|
||||
/** The timer for this request, created when the request started being processed */
|
||||
public RTimer getRequestTimer();
|
||||
|
||||
/** The index searcher associated with this request */
|
||||
public SolrIndexSearcher getSearcher();
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.solr.request;
|
|||
|
||||
import org.apache.solr.response.SolrQueryResponse;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.util.RTimer;
|
||||
import org.apache.solr.util.RefCounted;
|
||||
import org.apache.solr.schema.IndexSchema;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
|
@ -50,10 +51,17 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl
|
|||
protected Map<Object,Object> context;
|
||||
protected Iterable<ContentStream> streams;
|
||||
|
||||
public SolrQueryRequestBase(SolrCore core, SolrParams params) {
|
||||
private final RTimer requestTimer;
|
||||
|
||||
public SolrQueryRequestBase(SolrCore core, SolrParams params, RTimer requestTimer) {
|
||||
this.core = core;
|
||||
this.schema = null == core ? null : core.getLatestSchema();
|
||||
this.params = this.origParams = params;
|
||||
this.requestTimer = requestTimer;
|
||||
}
|
||||
|
||||
public SolrQueryRequestBase(SolrCore core, SolrParams params) {
|
||||
this(core, params, new RTimer());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -85,6 +93,10 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl
|
|||
return startTime;
|
||||
}
|
||||
|
||||
public RTimer getRequestTimer () {
|
||||
return requestTimer;
|
||||
}
|
||||
|
||||
// The index searcher associated with this request
|
||||
protected RefCounted<SolrIndexSearcher> searcherHolder;
|
||||
@Override
|
||||
|
|
|
@ -171,44 +171,7 @@ public class SolrQueryResponse {
|
|||
return err;
|
||||
}
|
||||
|
||||
/**
|
||||
* The endtime of the request in milliseconds.
|
||||
* Used to calculate query time.
|
||||
* @see #setEndTime(long)
|
||||
* @see #getEndTime()
|
||||
*/
|
||||
protected long endtime;
|
||||
|
||||
/**
|
||||
* Get the time in milliseconds when the response officially finished.
|
||||
*/
|
||||
public long getEndTime() {
|
||||
if (endtime==0) {
|
||||
setEndTime();
|
||||
}
|
||||
return endtime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Stop the timer for how long this query took.
|
||||
* @see #setEndTime(long)
|
||||
*/
|
||||
public long setEndTime() {
|
||||
return setEndTime(System.currentTimeMillis());
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the in milliseconds when the response officially finished.
|
||||
* @see #setEndTime()
|
||||
*/
|
||||
public long setEndTime(long endtime) {
|
||||
if (endtime!=0) {
|
||||
this.endtime=endtime;
|
||||
}
|
||||
return this.endtime;
|
||||
}
|
||||
|
||||
/** Repsonse header to be logged */
|
||||
/** Response header to be logged */
|
||||
public NamedList<Object> getResponseHeader() {
|
||||
@SuppressWarnings("unchecked")
|
||||
SimpleOrderedMap<Object> header = (SimpleOrderedMap<Object>) values.get("responseHeader");
|
||||
|
|
|
@ -70,6 +70,7 @@ import org.apache.solr.response.SolrQueryResponse;
|
|||
import org.apache.solr.servlet.cache.HttpCacheHeaderUtil;
|
||||
import org.apache.solr.servlet.cache.Method;
|
||||
import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
|
||||
import org.apache.solr.util.RTimer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -233,6 +234,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
|
|||
String corename = "";
|
||||
String origCorename = null;
|
||||
try {
|
||||
// set a request timer which can be reused by requests if needed
|
||||
req.setAttribute(SolrRequestParsers.REQUEST_TIMER_SERVLET_ATTRIBUTE, new RTimer());
|
||||
// put the core container in request attribute
|
||||
req.setAttribute("org.apache.solr.CoreContainer", cores);
|
||||
String path = req.getServletPath();
|
||||
|
|
|
@ -58,6 +58,7 @@ import org.apache.solr.core.SolrConfig;
|
|||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.request.SolrQueryRequestBase;
|
||||
import org.apache.solr.util.RTimer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -78,6 +79,8 @@ public class SolrRequestParsers
|
|||
public static final String INPUT_ENCODING_KEY = "ie";
|
||||
private static final byte[] INPUT_ENCODING_BYTES = INPUT_ENCODING_KEY.getBytes(CHARSET_US_ASCII);
|
||||
|
||||
public static final String REQUEST_TIMER_SERVLET_ATTRIBUTE = "org.apache.solr.RequestTimer";
|
||||
|
||||
private final HashMap<String, SolrRequestParser> parsers =
|
||||
new HashMap<>();
|
||||
private final boolean enableRemoteStreams;
|
||||
|
@ -137,6 +140,16 @@ public class SolrRequestParsers
|
|||
parsers.put( "", standard );
|
||||
}
|
||||
|
||||
private static RTimer getRequestTimer(HttpServletRequest req)
|
||||
{
|
||||
final Object reqTimer = req.getAttribute(REQUEST_TIMER_SERVLET_ATTRIBUTE);
|
||||
if (reqTimer != null && reqTimer instanceof RTimer) {
|
||||
return ((RTimer) reqTimer);
|
||||
}
|
||||
|
||||
return new RTimer();
|
||||
}
|
||||
|
||||
public SolrQueryRequest parse( SolrCore core, String path, HttpServletRequest req ) throws Exception
|
||||
{
|
||||
SolrRequestParser parser = standard;
|
||||
|
@ -146,7 +159,7 @@ public class SolrRequestParsers
|
|||
// Pick the parser from the request...
|
||||
ArrayList<ContentStream> streams = new ArrayList<>(1);
|
||||
SolrParams params = parser.parseParamsAndFillStreams( req, streams );
|
||||
SolrQueryRequest sreq = buildRequestFrom( core, params, streams );
|
||||
SolrQueryRequest sreq = buildRequestFrom( core, params, streams, getRequestTimer(req) );
|
||||
|
||||
// Handlers and login will want to know the path. If it contains a ':'
|
||||
// the handler could use it for RESTful URLs
|
||||
|
@ -160,6 +173,11 @@ public class SolrRequestParsers
|
|||
}
|
||||
|
||||
public SolrQueryRequest buildRequestFrom( SolrCore core, SolrParams params, Collection<ContentStream> streams ) throws Exception
|
||||
{
|
||||
return buildRequestFrom( core, params, streams, new RTimer() );
|
||||
}
|
||||
|
||||
private SolrQueryRequest buildRequestFrom( SolrCore core, SolrParams params, Collection<ContentStream> streams, RTimer requestTimer ) throws Exception
|
||||
{
|
||||
// The content type will be applied to all streaming content
|
||||
String contentType = params.get( CommonParams.STREAM_CONTENTTYPE );
|
||||
|
@ -206,7 +224,7 @@ public class SolrRequestParsers
|
|||
}
|
||||
}
|
||||
|
||||
SolrQueryRequestBase q = new SolrQueryRequestBase( core, params ) { };
|
||||
SolrQueryRequestBase q = new SolrQueryRequestBase( core, params, requestTimer ) { };
|
||||
if( streams != null && streams.size() > 0 ) {
|
||||
q.setContentStreams( streams );
|
||||
}
|
||||
|
|
|
@ -188,9 +188,9 @@ class LogUpdateProcessor extends UpdateRequestProcessor {
|
|||
log.info(getLogStringAndClearRspToLog());
|
||||
}
|
||||
|
||||
if (log.isWarnEnabled()) {
|
||||
long elapsed = rsp.getEndTime() - req.getStartTime();
|
||||
if (slowUpdateThresholdMillis >= 0 && elapsed >= slowUpdateThresholdMillis) {
|
||||
if (log.isWarnEnabled() && slowUpdateThresholdMillis >= 0) {
|
||||
final long elapsed = (long) req.getRequestTimer().getTime();
|
||||
if (elapsed >= slowUpdateThresholdMillis) {
|
||||
log.warn("slow: " + getLogStringAndClearRspToLog());
|
||||
}
|
||||
}
|
||||
|
@ -208,7 +208,7 @@ class LogUpdateProcessor extends UpdateRequestProcessor {
|
|||
if (deletes != null && numDeletes > maxNumToLog) {
|
||||
deletes.add("... (" + numDeletes + " deletes)");
|
||||
}
|
||||
long elapsed = rsp.getEndTime() - req.getStartTime();
|
||||
final long elapsed = (long) req.getRequestTimer().getTime();
|
||||
|
||||
sb.append(toLog).append(" 0 ").append(elapsed);
|
||||
return sb.toString();
|
||||
|
|
|
@ -88,13 +88,14 @@ public class RTimer {
|
|||
startTime = now();
|
||||
}
|
||||
|
||||
/** Get total elapsed time for this timer.
|
||||
*
|
||||
* Timer must be STOPped.
|
||||
*/
|
||||
/** Get total elapsed time for this timer. */
|
||||
public double getTime() {
|
||||
assert state == STOPPED;
|
||||
return time;
|
||||
if (state == STOPPED) return time;
|
||||
else if (state == PAUSED) return culmTime;
|
||||
else {
|
||||
assert state == STARTED;
|
||||
return culmTime + (now() - startTime);
|
||||
}
|
||||
}
|
||||
|
||||
/** Create new subtimer with given name
|
||||
|
@ -117,7 +118,7 @@ public class RTimer {
|
|||
|
||||
public NamedList asNamedList() {
|
||||
NamedList<Object> m = new SimpleOrderedMap<>();
|
||||
m.add( "time", time );
|
||||
m.add( "time", getTime() );
|
||||
if( children.size() > 0 ) {
|
||||
for( Map.Entry<String, RTimer> entry : children ) {
|
||||
m.add( entry.getKey(), entry.getValue().asNamedList() );
|
||||
|
|
|
@ -410,6 +410,7 @@ public class SolrRequestParserTest extends SolrTestCaseJ4 {
|
|||
v.add(entry.getValue());
|
||||
expect(request.getHeaders(entry.getKey())).andReturn(v.elements()).anyTimes();
|
||||
}
|
||||
expect(request.getAttribute(SolrRequestParsers.REQUEST_TIMER_SERVLET_ATTRIBUTE)).andReturn(null).anyTimes();
|
||||
replay(request);
|
||||
|
||||
SolrRequestParsers parsers = new SolrRequestParsers(h.getCore().getSolrConfig());
|
||||
|
@ -430,6 +431,7 @@ public class SolrRequestParserTest extends SolrTestCaseJ4 {
|
|||
expect(request.getContentType()).andReturn(null).anyTimes();
|
||||
expect(request.getQueryString()).andReturn(null).anyTimes();
|
||||
expect(request.getHeader(anyObject())).andReturn(null).anyTimes();
|
||||
expect(request.getAttribute(SolrRequestParsers.REQUEST_TIMER_SERVLET_ATTRIBUTE)).andReturn(null).anyTimes();
|
||||
replay(request);
|
||||
|
||||
SolrRequestParsers parsers = new SolrRequestParsers(h.getCore().getSolrConfig());
|
||||
|
|
|
@ -18,20 +18,28 @@
|
|||
package org.apache.solr.util;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.common.util.StrUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class TestUtils extends LuceneTestCase {
|
||||
public class TestUtils extends SolrTestCaseJ4 {
|
||||
|
||||
public void testJoin() {
|
||||
assertEquals("a|b|c", StrUtils.join(Arrays.asList("a","b","c"), '|'));
|
||||
|
@ -136,7 +144,7 @@ public class TestUtils extends LuceneTestCase {
|
|||
try {
|
||||
iter.remove();
|
||||
Assert.fail( "should be unsupported..." );
|
||||
} catch( UnsupportedOperationException ex ) {}
|
||||
} catch( UnsupportedOperationException ignored) {}
|
||||
}
|
||||
// the values should be bigger
|
||||
assertEquals( new Integer(10), map.get( "one" ) );
|
||||
|
@ -154,4 +162,43 @@ public class TestUtils extends LuceneTestCase {
|
|||
assertEquals( num, NumberUtils.SortableStr2long(sortable, 0, sortable.length() ) );
|
||||
assertEquals( Long.toString(num), NumberUtils.SortableStr2long(sortable) );
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNanoTimeSpeed()
|
||||
{
|
||||
final int maxNumThreads = 100;
|
||||
final int numIters = 1000;
|
||||
if (VERBOSE) log.info("testNanoTime: maxNumThreads = {}, numIters = {}", maxNumThreads, numIters);
|
||||
|
||||
final ExecutorService workers = Executors.newCachedThreadPool(new DefaultSolrThreadFactory("nanoTimeTestThread"));
|
||||
|
||||
for (int numThreads = 1; numThreads <= maxNumThreads; numThreads++) {
|
||||
List<Callable<Long>> tasks = new ArrayList<> ();
|
||||
for (int i = 0; i < numThreads; i ++) {
|
||||
tasks.add(new Callable<Long>() {
|
||||
@Override
|
||||
public Long call() {
|
||||
final long startTime = System.nanoTime();
|
||||
for (int i = 0; i < numIters; i++) {
|
||||
System.nanoTime();
|
||||
}
|
||||
return System.nanoTime() - startTime;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
try {
|
||||
List<Future<Long>> results = workers.invokeAll(tasks);
|
||||
long totalTime = 0;
|
||||
for (Future<Long> res : results) {
|
||||
totalTime += res.get();
|
||||
}
|
||||
long timePerIter = totalTime / (numIters * numThreads);
|
||||
assertTrue("Time taken for System.nanoTime is too high", timePerIter < 10000);
|
||||
if (VERBOSE) log.info("numThreads = {}, time_per_call = {}ns", numThreads, timePerIter);
|
||||
} catch (InterruptedException | ExecutionException ignored) {}
|
||||
}
|
||||
|
||||
ExecutorUtil.shutdownAndAwaitTermination(workers);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue