Skip to content

Commit 75a6e84

Browse files
soumitra-stgortiz
authored andcommitted
Improved metrics for server grpc query (apache#13177)
1 parent e525508 commit 75a6e84

File tree

4 files changed

+48
-6
lines changed

4 files changed

+48
-6
lines changed

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

+2
Original file line numberDiff line numberDiff line change
@@ -108,6 +108,8 @@ public enum ServerMeter implements AbstractMetrics.Meter {
108108
GRPC_QUERIES("grpcQueries", true),
109109
GRPC_BYTES_RECEIVED("grpcBytesReceived", true),
110110
GRPC_BYTES_SENT("grpcBytesSent", true),
111+
GRPC_TRANSPORT_READY("grpcTransport", true),
112+
GRPC_TRANSPORT_TERMINATED("grpcTransport", true),
111113

112114
NUM_SEGMENTS_PRUNED_INVALID("numSegmentsPrunedInvalid", false),
113115
NUM_SEGMENTS_PRUNED_BY_LIMIT("numSegmentsPrunedByLimit", false),

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

+1
Original file line numberDiff line numberDiff line change
@@ -53,6 +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", false, "Total execution time of a successful query over gRPC"),
5657
UPSERT_SNAPSHOT_TIME_MS("milliseconds", false, "Total time taken to take upsert table snapshot");
5758

5859
private final String _timerName;

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

+37-4
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,11 @@
1818
*/
1919
package org.apache.pinot.core.transport.grpc;
2020

21+
import io.grpc.Attributes;
22+
import io.grpc.Grpc;
2123
import io.grpc.Server;
2224
import io.grpc.ServerBuilder;
25+
import io.grpc.ServerTransportFilter;
2326
import io.grpc.Status;
2427
import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts;
2528
import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder;
@@ -33,12 +36,14 @@
3336
import java.util.concurrent.ConcurrentHashMap;
3437
import java.util.concurrent.ExecutorService;
3538
import java.util.concurrent.Executors;
39+
import java.util.concurrent.TimeUnit;
3640
import nl.altindag.ssl.SSLFactory;
3741
import org.apache.pinot.common.config.GrpcConfig;
3842
import org.apache.pinot.common.config.TlsConfig;
3943
import org.apache.pinot.common.datatable.DataTable;
4044
import org.apache.pinot.common.metrics.ServerMeter;
4145
import org.apache.pinot.common.metrics.ServerMetrics;
46+
import org.apache.pinot.common.metrics.ServerTimer;
4247
import org.apache.pinot.common.proto.PinotQueryServerGrpc;
4348
import org.apache.pinot.common.proto.Server.ServerRequest;
4449
import org.apache.pinot.common.proto.Server.ServerResponse;
@@ -72,19 +77,41 @@ public class GrpcQueryServer extends PinotQueryServerGrpc.PinotQueryServerImplBa
7277
private final AccessControl _accessControl;
7378
private final ServerQueryLogger _queryLogger = ServerQueryLogger.getInstance();
7479

80+
// Filter to keep track of gRPC connections.
81+
private class GrpcQueryTransportFilter extends ServerTransportFilter {
82+
@Override
83+
public Attributes transportReady(Attributes transportAttrs) {
84+
LOGGER.info("gRPC transportReady: REMOTE_ADDR {}",
85+
transportAttrs != null ? transportAttrs.get(Grpc.TRANSPORT_ATTR_REMOTE_ADDR) : "null");
86+
_serverMetrics.addMeteredGlobalValue(ServerMeter.GRPC_TRANSPORT_READY, 1);
87+
return super.transportReady(transportAttrs);
88+
}
89+
90+
@Override
91+
public void transportTerminated(Attributes transportAttrs) {
92+
// transportTerminated can be called without transportReady before it, e.g. handshake fails
93+
// So, don't emit metrics if transportAttrs is null
94+
if (transportAttrs != null) {
95+
LOGGER.info("gRPC transportTerminated: REMOTE_ADDR {}", transportAttrs.get(Grpc.TRANSPORT_ATTR_REMOTE_ADDR));
96+
_serverMetrics.addMeteredGlobalValue(ServerMeter.GRPC_TRANSPORT_TERMINATED, 1);
97+
}
98+
}
99+
}
100+
75101
public GrpcQueryServer(int port, GrpcConfig config, TlsConfig tlsConfig, QueryExecutor queryExecutor,
76102
ServerMetrics serverMetrics, AccessControl accessControl) {
77103
_queryExecutor = queryExecutor;
78104
_serverMetrics = serverMetrics;
79105
if (tlsConfig != null) {
80106
try {
81107
_server = NettyServerBuilder.forPort(port).sslContext(buildGRpcSslContext(tlsConfig))
82-
.maxInboundMessageSize(config.getMaxInboundMessageSizeBytes()).addService(this).build();
108+
.maxInboundMessageSize(config.getMaxInboundMessageSizeBytes()).addService(this)
109+
.addTransportFilter(new GrpcQueryTransportFilter()).build();
83110
} catch (Exception e) {
84111
throw new RuntimeException("Failed to start secure grpcQueryServer", e);
85112
}
86113
} else {
87-
_server = ServerBuilder.forPort(port).addService(this).build();
114+
_server = ServerBuilder.forPort(port).addService(this).addTransportFilter(new GrpcQueryTransportFilter()).build();
88115
}
89116
_accessControl = accessControl;
90117
LOGGER.info("Initialized GrpcQueryServer on port: {} with numWorkerThreads: {}", port,
@@ -136,6 +163,7 @@ public void shutdown() {
136163

137164
@Override
138165
public void submit(ServerRequest request, StreamObserver<ServerResponse> responseObserver) {
166+
long startTime = System.nanoTime();
139167
_serverMetrics.addMeteredGlobalValue(ServerMeter.GRPC_QUERIES, 1);
140168
_serverMetrics.addMeteredGlobalValue(ServerMeter.GRPC_BYTES_RECEIVED, request.getSerializedSize());
141169

@@ -162,13 +190,16 @@ public void submit(ServerRequest request, StreamObserver<ServerResponse> respons
162190
_serverMetrics.addMeteredGlobalValue(ServerMeter.NO_TABLE_ACCESS, 1);
163191
responseObserver.onError(
164192
Status.NOT_FOUND.withDescription(exceptionMsg).withCause(unsupportedOperationException).asException());
193+
return;
165194
}
166195

167196
// Process the query
168197
InstanceResponseBlock instanceResponse;
169198
try {
170-
instanceResponse =
171-
_queryExecutor.execute(queryRequest, _executorService, new GrpcResultsBlockStreamer(responseObserver));
199+
LOGGER.info("Executing gRPC query request {}: {} received from broker: {}", queryRequest.getRequestId(),
200+
queryRequest.getQueryContext(), queryRequest.getBrokerId());
201+
instanceResponse = _queryExecutor.execute(queryRequest, _executorService,
202+
new GrpcResultsBlockStreamer(responseObserver, _serverMetrics));
172203
} catch (Exception e) {
173204
LOGGER.error("Caught exception while processing request {}: {} from broker: {}", queryRequest.getRequestId(),
174205
queryRequest.getQueryContext(), queryRequest.getBrokerId(), e);
@@ -192,6 +223,8 @@ public void submit(ServerRequest request, StreamObserver<ServerResponse> respons
192223
responseObserver.onNext(serverResponse);
193224
_serverMetrics.addMeteredGlobalValue(ServerMeter.GRPC_BYTES_SENT, serverResponse.getSerializedSize());
194225
responseObserver.onCompleted();
226+
_serverMetrics.addTimedTableValue(queryRequest.getTableNameWithType(), ServerTimer.GRPC_QUERY_EXECUTION_MS,
227+
System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
195228

196229
// Log the query
197230
if (_queryLogger != null) {

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

+8-2
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,8 @@
2323
import java.io.IOException;
2424
import java.util.Collection;
2525
import org.apache.pinot.common.datatable.DataTable;
26+
import org.apache.pinot.common.metrics.ServerMeter;
27+
import org.apache.pinot.common.metrics.ServerMetrics;
2628
import org.apache.pinot.common.proto.Server;
2729
import org.apache.pinot.common.utils.DataSchema;
2830
import org.apache.pinot.core.operator.blocks.results.BaseResultsBlock;
@@ -32,9 +34,11 @@
3234

3335
public class GrpcResultsBlockStreamer implements ResultsBlockStreamer {
3436
private final StreamObserver<Server.ServerResponse> _streamObserver;
37+
private final ServerMetrics _serverMetrics;
3538

36-
public GrpcResultsBlockStreamer(StreamObserver<Server.ServerResponse> streamObserver) {
39+
public GrpcResultsBlockStreamer(StreamObserver<Server.ServerResponse> streamObserver, ServerMetrics serverMetrics) {
3740
_streamObserver = streamObserver;
41+
_serverMetrics = serverMetrics;
3842
}
3943

4044
@Override
@@ -44,6 +48,8 @@ public void send(BaseResultsBlock block)
4448
Collection<Object[]> rows = block.getRows();
4549
Preconditions.checkState(dataSchema != null && rows != null, "Malformed data block");
4650
DataTable dataTable = block.getDataTable();
47-
_streamObserver.onNext(StreamingResponseUtils.getDataResponse(dataTable));
51+
Server.ServerResponse response = StreamingResponseUtils.getDataResponse(dataTable);
52+
_streamObserver.onNext(response);
53+
_serverMetrics.addMeteredGlobalValue(ServerMeter.GRPC_BYTES_SENT, response.getSerializedSize());
4854
}
4955
}

0 commit comments

Comments
 (0)