mirror of https://github.com/apache/druid.git
Introduce query/timeout/count metric (#10567)
* Add timeout metric * Add tests
This commit is contained in:
parent
3cafd531de
commit
111b431c07
|
@ -57,7 +57,8 @@ Available Metrics
|
|||
|`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/failed/count`|number of failed queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/interrupted/count`|number of queries interrupted due to cancellation or timeout|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/interrupted/count`|number of queries interrupted due to cancellation.|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/timeout/count`|number of timed out queries.|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`sqlQuery/time`|Milliseconds taken to complete a SQL query.|id, nativeQueryIds, dataSource, remoteAddress, success.|< 1s|
|
||||
|`sqlQuery/bytes`|number of bytes returned in SQL query response.|id, nativeQueryIds, dataSource, remoteAddress, success.| |
|
||||
|
||||
|
@ -74,7 +75,8 @@ Available Metrics
|
|||
|`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/failed/count`|number of failed queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/interrupted/count`|number of queries interrupted due to cancellation or timeout|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/interrupted/count`|number of queries interrupted due to cancellation.|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/timeout/count`|number of timed out queries.|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|
||||
### Real-time
|
||||
|
||||
|
@ -86,7 +88,8 @@ Available Metrics
|
|||
|`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/failed/count`|number of failed queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/interrupted/count`|number of queries interrupted due to cancellation or timeout|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/interrupted/count`|number of queries interrupted due to cancellation.|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/timeout/count`|number of timed out queries.|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|
||||
### Jetty
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
"query/success/count": [],
|
||||
"query/failed/count": [],
|
||||
"query/interrupted/count": [],
|
||||
"query/timeout/count": [],
|
||||
"query/wait/time": [
|
||||
"segment"
|
||||
],
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
"query/success/count" : { "dimensions" : [], "type" : "count" },
|
||||
"query/failed/count" : { "dimensions" : [], "type" : "count" },
|
||||
"query/interrupted/count" : { "dimensions" : [], "type" : "count" },
|
||||
"query/timeout/count" : { "dimensions" : [], "type" : "count" },
|
||||
|
||||
"query/cache/delta/numEntries" : { "dimensions" : [], "type" : "count" },
|
||||
"query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "count" },
|
||||
|
|
|
@ -425,6 +425,14 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet implements Qu
|
|||
return interruptedQueryCount.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimedOutQueryCount()
|
||||
{
|
||||
// Query timeout metric is not relevant here and this metric is already being tracked in the Broker and the
|
||||
// data nodes using QueryResource
|
||||
return 0L;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static String getAvaticaConnectionId(Map<String, Object> requestMap)
|
||||
{
|
||||
|
|
|
@ -108,6 +108,7 @@ public class QueryResource implements QueryCountStatsProvider
|
|||
private final AtomicLong successfulQueryCount = new AtomicLong();
|
||||
private final AtomicLong failedQueryCount = new AtomicLong();
|
||||
private final AtomicLong interruptedQueryCount = new AtomicLong();
|
||||
private final AtomicLong timedOutQueryCount = new AtomicLong();
|
||||
|
||||
@Inject
|
||||
public QueryResource(
|
||||
|
@ -332,7 +333,7 @@ public class QueryResource implements QueryCountStatsProvider
|
|||
return ioReaderWriter.gotError(e);
|
||||
}
|
||||
catch (QueryTimeoutException timeout) {
|
||||
interruptedQueryCount.incrementAndGet();
|
||||
timedOutQueryCount.incrementAndGet();
|
||||
queryLifecycle.emitLogsAndMetrics(timeout, req.getRemoteAddr(), -1);
|
||||
return ioReaderWriter.gotTimeout(timeout);
|
||||
}
|
||||
|
@ -514,4 +515,10 @@ public class QueryResource implements QueryCountStatsProvider
|
|||
{
|
||||
return interruptedQueryCount.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimedOutQueryCount()
|
||||
{
|
||||
return timedOutQueryCount.get();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,13 +48,16 @@ public class QueryCountStatsMonitor extends AbstractMonitor
|
|||
final long successfulQueryCount = statsProvider.getSuccessfulQueryCount();
|
||||
final long failedQueryCount = statsProvider.getFailedQueryCount();
|
||||
final long interruptedQueryCount = statsProvider.getInterruptedQueryCount();
|
||||
final long timedOutQueryCount = statsProvider.getTimedOutQueryCount();
|
||||
|
||||
Map<String, Long> diff = keyedDiff.to(
|
||||
"queryCountStats",
|
||||
ImmutableMap.of(
|
||||
"query/count", successfulQueryCount + failedQueryCount + interruptedQueryCount,
|
||||
"query/count", successfulQueryCount + failedQueryCount + interruptedQueryCount + timedOutQueryCount,
|
||||
"query/success/count", successfulQueryCount,
|
||||
"query/failed/count", failedQueryCount,
|
||||
"query/interrupted/count", interruptedQueryCount
|
||||
"query/interrupted/count", interruptedQueryCount,
|
||||
"query/timeout/count", timedOutQueryCount
|
||||
)
|
||||
);
|
||||
if (diff != null) {
|
||||
|
|
|
@ -21,9 +21,23 @@ package org.apache.druid.server.metrics;
|
|||
|
||||
public interface QueryCountStatsProvider
|
||||
{
|
||||
/**
|
||||
* Returns the number of successful queries processed during the emission period.
|
||||
*/
|
||||
long getSuccessfulQueryCount();
|
||||
|
||||
/**
|
||||
* Returns the number of failed queries during the emission period.
|
||||
*/
|
||||
long getFailedQueryCount();
|
||||
|
||||
/**
|
||||
* Returns the number of queries interrupted due to cancellation during the emission period.
|
||||
*/
|
||||
long getInterruptedQueryCount();
|
||||
|
||||
/**
|
||||
* Returns the number of timed out queries during the emission period.
|
||||
*/
|
||||
long getTimedOutQueryCount();
|
||||
}
|
||||
|
|
|
@ -684,6 +684,8 @@ public class QueryResourceTest
|
|||
}
|
||||
Assert.assertEquals("Query Timed Out!", ex.getMessage());
|
||||
Assert.assertEquals(QueryTimeoutException.ERROR_CODE, ex.getErrorCode());
|
||||
Assert.assertEquals(1, timeoutQueryResource.getTimedOutQueryCount());
|
||||
|
||||
}
|
||||
|
||||
@Test(timeout = 60_000L)
|
||||
|
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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.druid.server.metrics;
|
||||
|
||||
import org.apache.druid.java.util.metrics.StubServiceEmitter;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class QueryCountStatsMonitorTest
|
||||
{
|
||||
private QueryCountStatsProvider queryCountStatsProvider;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
queryCountStatsProvider = new QueryCountStatsProvider()
|
||||
{
|
||||
private long successEmitCount = 0;
|
||||
private long failedEmitCount = 0;
|
||||
private long interruptedEmitCount = 0;
|
||||
private long timedOutEmitCount = 0;
|
||||
|
||||
@Override
|
||||
public long getSuccessfulQueryCount()
|
||||
{
|
||||
successEmitCount += 1;
|
||||
return successEmitCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFailedQueryCount()
|
||||
{
|
||||
failedEmitCount += 2;
|
||||
return failedEmitCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInterruptedQueryCount()
|
||||
{
|
||||
interruptedEmitCount += 3;
|
||||
return interruptedEmitCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimedOutQueryCount()
|
||||
{
|
||||
timedOutEmitCount += 4;
|
||||
return timedOutEmitCount;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMonitor()
|
||||
{
|
||||
final QueryCountStatsMonitor monitor = new QueryCountStatsMonitor(queryCountStatsProvider);
|
||||
final StubServiceEmitter emitter = new StubServiceEmitter("service", "host");
|
||||
monitor.doMonitor(emitter);
|
||||
// Trigger metric emission
|
||||
monitor.doMonitor(emitter);
|
||||
Map<String, Long> resultMap = emitter.getEvents()
|
||||
.stream()
|
||||
.collect(Collectors.toMap(
|
||||
event -> (String) event.toMap().get("metric"),
|
||||
event -> (Long) event.toMap().get("value")
|
||||
));
|
||||
Assert.assertEquals(5, resultMap.size());
|
||||
Assert.assertEquals(1L, (long) resultMap.get("query/success/count"));
|
||||
Assert.assertEquals(2L, (long) resultMap.get("query/failed/count"));
|
||||
Assert.assertEquals(3L, (long) resultMap.get("query/interrupted/count"));
|
||||
Assert.assertEquals(4L, (long) resultMap.get("query/timeout/count"));
|
||||
Assert.assertEquals(10L, (long) resultMap.get("query/count"));
|
||||
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue