gortiz commented on code in PR #15609:
URL: https://github.com/apache/pinot/pull/15609#discussion_r2068639112
##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java:
##########
@@ -146,18 +162,40 @@ public QueryResult submitAndReduce(RequestContext
context, DispatchableSubPlan d
Set<QueryServerInstance> servers = new HashSet<>();
try {
submit(requestId, dispatchableSubPlan, timeoutMs, servers, queryOptions);
- try {
- return runReducer(requestId, dispatchableSubPlan, timeoutMs,
queryOptions, _mailboxService);
- } finally {
- if (isQueryCancellationEnabled()) {
- _serversByQuery.remove(requestId);
- }
- }
+ return runReducer(requestId, dispatchableSubPlan, timeoutMs,
queryOptions, _mailboxService);
+ } catch (Exception ex) {
+ return tryRecover(context.getRequestId(), servers, ex);
} catch (Throwable e) {
// TODO: Consider always cancel when it returns (early terminate)
- cancel(requestId, servers);
+ cancel(requestId);
throw e;
+ } finally {
+ if (isQueryCancellationEnabled()) {
+ _serversByQuery.remove(requestId);
+ }
+ }
+ }
+
+ private QueryResult tryRecover(long requestId, Set<QueryServerInstance>
servers, Exception ex)
+ throws Exception {
+ if (servers.isEmpty()) {
+ throw ex;
+ }
+ QueryErrorCode errorCode;
+ if (ex instanceof TimeoutException) {
+ errorCode = QueryErrorCode.EXECUTION_TIMEOUT;
+ } else if (ex instanceof QueryException) {
+ errorCode = ((QueryException) ex).getErrorCode();
+ } else {
+ cancel(requestId, servers);
+ throw ex;
+ }
+ MultiStageQueryStats stats = cancelWithStats(requestId, servers);
Review Comment:
I think the correct point of view is the opposite. It is not that timeouts
require special handling. We will also treat QueryExceptions in the same manner
and that will be the standard behavior on standard failures.
The special handling is in the other case: For unknown exceptions, we will
cancel without statistics because we do not actually know what happened.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]