18
18
*/
19
19
package org .apache .pinot .broker .requesthandler ;
20
20
21
+ import io .grpc .ConnectivityState ;
21
22
import java .util .HashMap ;
22
23
import java .util .Iterator ;
23
24
import java .util .List ;
31
32
import org .apache .pinot .broker .routing .BrokerRoutingManager ;
32
33
import org .apache .pinot .common .config .GrpcConfig ;
33
34
import org .apache .pinot .common .config .provider .TableCache ;
35
+ import org .apache .pinot .common .failuredetector .FailureDetector ;
34
36
import org .apache .pinot .common .proto .Server ;
35
37
import org .apache .pinot .common .request .BrokerRequest ;
36
38
import org .apache .pinot .common .response .broker .BrokerResponseNative ;
43
45
import org .apache .pinot .spi .config .table .TableType ;
44
46
import org .apache .pinot .spi .env .PinotConfiguration ;
45
47
import org .apache .pinot .spi .trace .RequestContext ;
48
+ import org .slf4j .Logger ;
49
+ import org .slf4j .LoggerFactory ;
46
50
47
51
48
52
/**
49
53
* The <code>GrpcBrokerRequestHandler</code> class communicates query request via GRPC.
50
54
*/
51
55
@ ThreadSafe
52
56
public class GrpcBrokerRequestHandler extends BaseSingleStageBrokerRequestHandler {
57
+ private static final Logger LOGGER = LoggerFactory .getLogger (GrpcBrokerRequestHandler .class );
58
+
53
59
private final StreamingReduceService _streamingReduceService ;
54
60
private final PinotStreamingQueryClient _streamingQueryClient ;
61
+ private final FailureDetector _failureDetector ;
55
62
56
63
// TODO: Support TLS
57
64
public GrpcBrokerRequestHandler (PinotConfiguration config , String brokerId , BrokerRoutingManager routingManager ,
58
- AccessControlFactory accessControlFactory , QueryQuotaManager queryQuotaManager , TableCache tableCache ) {
65
+ AccessControlFactory accessControlFactory , QueryQuotaManager queryQuotaManager , TableCache tableCache ,
66
+ FailureDetector failureDetector ) {
59
67
super (config , brokerId , routingManager , accessControlFactory , queryQuotaManager , tableCache );
60
68
_streamingReduceService = new StreamingReduceService (config );
61
69
_streamingQueryClient = new PinotStreamingQueryClient (GrpcConfig .buildGrpcQueryConfig (config ));
70
+ _failureDetector = failureDetector ;
71
+ _failureDetector .registerUnhealthyServerRetrier (this ::retryUnhealthyServer );
62
72
}
63
73
64
74
@ Override
@@ -81,7 +91,6 @@ protected BrokerResponseNative processBrokerRequest(long requestId, BrokerReques
81
91
@ Nullable Map <ServerInstance , ServerRouteInfo > realtimeRoutingTable , long timeoutMs ,
82
92
ServerStats serverStats , RequestContext requestContext )
83
93
throws Exception {
84
- // TODO: Support failure detection
85
94
// TODO: Add servers queried/responded stats
86
95
assert offlineBrokerRequest != null || realtimeBrokerRequest != null ;
87
96
Map <ServerRoutingInstance , Iterator <Server .ServerResponse >> responseMap = new HashMap <>();
@@ -112,33 +121,39 @@ private void sendRequest(long requestId, TableType tableType, BrokerRequest brok
112
121
ServerInstance serverInstance = routingEntry .getKey ();
113
122
// TODO: support optional segments for GrpcQueryServer.
114
123
List <String > segments = routingEntry .getValue ().getSegments ();
115
- String serverHost = serverInstance .getHostname ();
116
- int port = serverInstance .getGrpcPort ();
117
124
// TODO: enable throttling on per host bases.
118
- Iterator <Server .ServerResponse > streamingResponse = _streamingQueryClient .submit (serverHost , port ,
119
- new GrpcRequestBuilder ().setRequestId (requestId ).setBrokerId (_brokerId ).setEnableTrace (trace )
120
- .setEnableStreaming (true ).setBrokerRequest (brokerRequest ).setSegments (segments ).build ());
121
- responseMap .put (serverInstance .toServerRoutingInstance (tableType , ServerInstance .RoutingType .GRPC ),
122
- streamingResponse );
125
+ try {
126
+ Iterator <Server .ServerResponse > streamingResponse = _streamingQueryClient .submit (serverInstance ,
127
+ new GrpcRequestBuilder ().setRequestId (requestId ).setBrokerId (_brokerId ).setEnableTrace (trace )
128
+ .setEnableStreaming (true ).setBrokerRequest (brokerRequest ).setSegments (segments ).build ());
129
+ responseMap .put (serverInstance .toServerRoutingInstance (tableType , ServerInstance .RoutingType .GRPC ),
130
+ streamingResponse );
131
+ } catch (Exception e ) {
132
+ LOGGER .warn ("Failed to send request {} to server: {}" , requestId , serverInstance .getInstanceId (), e );
133
+ _failureDetector .markServerUnhealthy (serverInstance .getInstanceId ());
134
+ }
123
135
}
124
136
}
125
137
126
138
public static class PinotStreamingQueryClient {
127
139
private final Map <String , GrpcQueryClient > _grpcQueryClientMap = new ConcurrentHashMap <>();
140
+ private final Map <String , String > _instanceIdToHostnamePortMap = new ConcurrentHashMap <>();
128
141
private final GrpcConfig _config ;
129
142
130
143
public PinotStreamingQueryClient (GrpcConfig config ) {
131
144
_config = config ;
132
145
}
133
146
134
- public Iterator <Server .ServerResponse > submit (String host , int port , Server .ServerRequest serverRequest ) {
135
- GrpcQueryClient client = getOrCreateGrpcQueryClient (host , port );
147
+ public Iterator <Server .ServerResponse > submit (ServerInstance serverInstance , Server .ServerRequest serverRequest ) {
148
+ GrpcQueryClient client = getOrCreateGrpcQueryClient (serverInstance );
136
149
return client .submit (serverRequest );
137
150
}
138
151
139
- private GrpcQueryClient getOrCreateGrpcQueryClient (String host , int port ) {
140
- String key = String .format ("%s_%d" , host , port );
141
- return _grpcQueryClientMap .computeIfAbsent (key , k -> new GrpcQueryClient (host , port , _config ));
152
+ private GrpcQueryClient getOrCreateGrpcQueryClient (ServerInstance serverInstance ) {
153
+ String hostnamePort = String .format ("%s_%d" , serverInstance .getHostname (), serverInstance .getGrpcPort ());
154
+ _instanceIdToHostnamePortMap .put (serverInstance .getInstanceId (), hostnamePort );
155
+ return _grpcQueryClientMap .computeIfAbsent (hostnamePort ,
156
+ k -> new GrpcQueryClient (serverInstance .getHostname (), serverInstance .getGrpcPort (), _config ));
142
157
}
143
158
144
159
public void shutdown () {
@@ -147,4 +162,26 @@ public void shutdown() {
147
162
}
148
163
}
149
164
}
165
+
166
+ /**
167
+ * Check if a server that was previously detected as unhealthy is now healthy.
168
+ */
169
+ private boolean retryUnhealthyServer (String instanceId ) {
170
+ LOGGER .info ("Checking gRPC connection to unhealthy server: {}" , instanceId );
171
+ ServerInstance serverInstance = _routingManager .getEnabledServerInstanceMap ().get (instanceId );
172
+ if (serverInstance == null ) {
173
+ LOGGER .info ("Failed to find enabled server: {} in routing manager, skipping the retry" , instanceId );
174
+ return false ;
175
+ }
176
+
177
+ String hostnamePort = _streamingQueryClient ._instanceIdToHostnamePortMap .get (instanceId );
178
+ GrpcQueryClient client = _streamingQueryClient ._grpcQueryClientMap .get (hostnamePort );
179
+
180
+ if (client == null ) {
181
+ LOGGER .warn ("No GrpcQueryClient found for server with instanceId: {}" , instanceId );
182
+ return false ;
183
+ }
184
+
185
+ return client .getChannel ().getState (true ) == ConnectivityState .READY ;
186
+ }
150
187
}
0 commit comments