dang-stripe opened a new pull request, #17466:
URL: https://github.com/apache/pinot/pull/17466

   ## Problem
   
   This fixes https://github.com/apache/pinot/issues/17465 where MSE queries 
can fail when a server is re-enabled after being excluded by the failure 
detector.
   
   If a downstream server is uncleanly killed, the GRPC channel used by the 
broker's DispatchClient can go into a `TRANSIENT_FAILURE` state which will fast 
fail requests while attempting reconnection w/ exponential backoff. The default 
max backoff for GRPC is ~2 minutes so we can see query failures for that 
duration once the server comes up healthy again and is re-added to the broker 
routing map until the GRPC channel retries the connection.
   
   When the failure detector checks connectivity of the instance through the 
dispatch client with 
[`client.getChannel().getState(true)`](https://github.com/apache/pinot/blob/7ca198408ce254da73a4a8ba1ae955ee95640715/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java#L278C38-L278C65),
 it attempts reconnection [when the channel state is 
`IDLE`](https://grpc.github.io/grpc-java/javadoc/io/grpc/ManagedChannel.html#getState(boolean)),
 so it doesn't handle this case.
   
   ## Solution
   
   This adds a server re-enable callback to the BrokerRoutingManager that will 
reset the dispatch client used for that server. This will create a new 
ManagedChannel bypassing any ongoing reconnection backoff happening at the time.
   
   I considered several other solutions here as well:
   - Don't remove the server from `_excludedServers` in BrokerRoutingManager - 
This means even after the server comes up healthy, the broker may end up 
waiting ~2 minutes for the channel to reconnect and the failure detector to 
detect that the channel is healthy again before the server is added back to 
routing. This seemed unnecessarily long, but is a simpler change.
   - Customize the GRPC reconnection options - We could tune the GRPC backoff 
policy to shorten the time query failures can happen. This wouldn't fully 
eliminate the issue though.
   
   ## Testing
   
   I tested this by deploying it to one of our clusters, running a load test, 
killing a server, and monitoring query failures. I no longer see the same query 
failure spike when the server comes up healthy.
   
   cc @Jackie-Jiang @yashmayya @jadami10 


-- 
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]

Reply via email to