mirror of https://github.com/apache/druid.git
Fix query failed metric double count bug (#17454)
This commit is contained in:
parent
0dcc2bc469
commit
f906d0d446
|
@ -229,24 +229,8 @@ public abstract class QueryResultPusher
|
|||
return handleDruidException(resultsWriter, DruidException.fromFailure(new QueryExceptionCompat(e)));
|
||||
}
|
||||
|
||||
private Response handleDruidException(ResultsWriter resultsWriter, DruidException e)
|
||||
private void incrementQueryCounterForException(final DruidException e)
|
||||
{
|
||||
if (resultsWriter != null) {
|
||||
resultsWriter.recordFailure(e);
|
||||
counter.incrementFailed();
|
||||
|
||||
if (accumulator != null && accumulator.isInitialized()) {
|
||||
// We already started sending a response when we got the error message. In this case we just give up
|
||||
// and hope that the partial stream generates a meaningful failure message for our client. We could consider
|
||||
// also throwing the exception body into the response to make it easier for the client to choke if it manages
|
||||
// to parse a meaningful object out, but that's potentially an API change so we leave that as an exercise for
|
||||
// the future.
|
||||
trailerFields.put(QueryResource.ERROR_MESSAGE_TRAILER_HEADER, e.getMessage());
|
||||
trailerFields.put(QueryResource.RESPONSE_COMPLETE_TRAILER_HEADER, "false");
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
switch (e.getCategory()) {
|
||||
case INVALID_INPUT:
|
||||
case UNAUTHORIZED:
|
||||
|
@ -264,6 +248,26 @@ public abstract class QueryResultPusher
|
|||
counter.incrementTimedOut();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
private Response handleDruidException(ResultsWriter resultsWriter, DruidException e)
|
||||
{
|
||||
incrementQueryCounterForException(e);
|
||||
|
||||
if (resultsWriter != null) {
|
||||
resultsWriter.recordFailure(e);
|
||||
|
||||
if (accumulator != null && accumulator.isInitialized()) {
|
||||
// We already started sending a response when we got the error message. In this case we just give up
|
||||
// and hope that the partial stream generates a meaningful failure message for our client. We could consider
|
||||
// also throwing the exception body into the response to make it easier for the client to choke if it manages
|
||||
// to parse a meaningful object out, but that's potentially an API change so we leave that as an exercise for
|
||||
// the future.
|
||||
trailerFields.put(QueryResource.ERROR_MESSAGE_TRAILER_HEADER, e.getMessage());
|
||||
trailerFields.put(QueryResource.RESPONSE_COMPLETE_TRAILER_HEADER, "false");
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
if (response == null) {
|
||||
final Response.ResponseBuilder bob = Response
|
||||
|
|
|
@ -1245,6 +1245,8 @@ public class QueryResourceTest
|
|||
for (Future<Boolean> theFuture : back2) {
|
||||
Assert.assertTrue(theFuture.get());
|
||||
}
|
||||
Assert.assertEquals(2, queryResource.getSuccessfulQueryCount());
|
||||
Assert.assertEquals(1, queryResource.getFailedQueryCount());
|
||||
}
|
||||
|
||||
@Test(timeout = 10_000L)
|
||||
|
|
Loading…
Reference in New Issue