Skip to content

Commit c702e3a

Browse files
committed
Removed failed execution timer, log request before execution, and changed execution time at table level.
1 parent bfb0071 commit c702e3a

File tree

2 files changed

+5
-12
lines changed

2 files changed

+5
-12
lines changed

pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerTimer.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -53,8 +53,7 @@ public enum ServerTimer implements AbstractMetrics.Timer {
5353
"Total time taken to preload a table partition of an upsert table with upsert snapshot"),
5454
UPSERT_REMOVE_EXPIRED_PRIMARY_KEYS_TIME_MS("milliseconds", false,
5555
"Total time taken to delete expired primary keys based on metadataTTL or deletedKeysTTL"),
56-
GRPC_QUERY_EXECUTION_MS("milliseconds", true, "Total execution time of a successful query over gRPC"),
57-
GRPC_FAILED_QUERY_EXECUTION_MS("milliseconds", true, "Total execution time of a failing query over gRPC"),
56+
GRPC_QUERY_EXECUTION_MS("milliseconds", false, "Total execution time of a successful query over gRPC"),
5857
UPSERT_SNAPSHOT_TIME_MS("milliseconds", false, "Total time taken to take upsert table snapshot");
5958

6059
private final String _timerName;

pinot-core/src/main/java/org/apache/pinot/core/transport/grpc/GrpcQueryServer.java

+4-10
Original file line numberDiff line numberDiff line change
@@ -172,8 +172,6 @@ public void submit(ServerRequest request, StreamObserver<ServerResponse> respons
172172
LOGGER.error("Caught exception while deserializing the request: {}", request, e);
173173
_serverMetrics.addMeteredGlobalValue(ServerMeter.REQUEST_DESERIALIZATION_EXCEPTIONS, 1);
174174
responseObserver.onError(Status.INVALID_ARGUMENT.withDescription("Bad request").withCause(e).asException());
175-
_serverMetrics.addTimedValue(ServerTimer.GRPC_FAILED_QUERY_EXECUTION_MS, System.nanoTime() - startTime,
176-
TimeUnit.NANOSECONDS);
177175
return;
178176
}
179177

@@ -189,23 +187,21 @@ public void submit(ServerRequest request, StreamObserver<ServerResponse> respons
189187
_serverMetrics.addMeteredGlobalValue(ServerMeter.NO_TABLE_ACCESS, 1);
190188
responseObserver.onError(
191189
Status.NOT_FOUND.withDescription(exceptionMsg).withCause(unsupportedOperationException).asException());
192-
_serverMetrics.addTimedValue(ServerTimer.GRPC_FAILED_QUERY_EXECUTION_MS, System.nanoTime() - startTime,
193-
TimeUnit.NANOSECONDS);
194190
return;
195191
}
196192

197193
// Process the query
198194
InstanceResponseBlock instanceResponse;
199195
try {
196+
LOGGER.info("Executing gRPC query request {}: {} received from broker: {}", queryRequest.getRequestId(),
197+
queryRequest.getQueryContext(), queryRequest.getBrokerId());
200198
instanceResponse = _queryExecutor.execute(queryRequest, _executorService,
201199
new GrpcResultsBlockStreamer(responseObserver, _serverMetrics));
202200
} catch (Exception e) {
203201
LOGGER.error("Caught exception while processing request {}: {} from broker: {}", queryRequest.getRequestId(),
204202
queryRequest.getQueryContext(), queryRequest.getBrokerId(), e);
205203
_serverMetrics.addMeteredGlobalValue(ServerMeter.UNCAUGHT_EXCEPTIONS, 1);
206204
responseObserver.onError(Status.INTERNAL.withCause(e).asException());
207-
_serverMetrics.addTimedValue(ServerTimer.GRPC_FAILED_QUERY_EXECUTION_MS, System.nanoTime() - startTime,
208-
TimeUnit.NANOSECONDS);
209205
return;
210206
}
211207

@@ -219,15 +215,13 @@ public void submit(ServerRequest request, StreamObserver<ServerResponse> respons
219215
queryRequest.getRequestId(), queryRequest.getQueryContext(), queryRequest.getBrokerId(), e);
220216
_serverMetrics.addMeteredGlobalValue(ServerMeter.RESPONSE_SERIALIZATION_EXCEPTIONS, 1);
221217
responseObserver.onError(Status.INTERNAL.withCause(e).asException());
222-
_serverMetrics.addTimedValue(ServerTimer.GRPC_FAILED_QUERY_EXECUTION_MS, System.nanoTime() - startTime,
223-
TimeUnit.NANOSECONDS);
224218
return;
225219
}
226220
responseObserver.onNext(serverResponse);
227221
_serverMetrics.addMeteredGlobalValue(ServerMeter.GRPC_BYTES_SENT, serverResponse.getSerializedSize());
228222
responseObserver.onCompleted();
229-
_serverMetrics.addTimedValue(ServerTimer.GRPC_QUERY_EXECUTION_MS, System.nanoTime() - startTime,
230-
TimeUnit.NANOSECONDS);
223+
_serverMetrics.addTimedTableValue(queryRequest.getTableNameWithType(), ServerTimer.GRPC_QUERY_EXECUTION_MS,
224+
System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
231225

232226
// Log the query
233227
if (_queryLogger != null) {

0 commit comments

Comments
 (0)