dang-stripe commented on issue #15997: URL: https://github.com/apache/pinot/issues/15997#issuecomment-3673941006
I've done some investigation and think the failures we saw are specific to the failure detector + MSE + GRPC reconnection behavior. By default, the GRPC uses an exponential backoff policy w/ a 2 minute cap to retry the connection for ManagedChannels. In https://github.com/apache/pinot/blob/master/pinot-broker/src/main/java/org/apache/pinot/broker/routing/BrokerRoutingManager.java#L387-L390, we remove the excluded server if it comes up healthy, independently of whether the underlying GRPC connection is in a READY state. If this happens before the the channel has had a chance to establish the connection again, then queries will fail. Here are the failure detector logs to illustrate this: ``` # server marked unhealthy after kill -9 2025-12-17 23:43:02.619 WARN [BaseExponentialBackoffRetryFailureDetector] [jersey-server-managed-async-executor-309:981] Mark server: Server_st6 as unhealthy # first grpc connection check (i've skipped the rest to keep it brief) 2025-12-17 23:43:07.710 INFO [BaseExponentialBackoffRetryFailureDetector] [failure-detector-retry:172] Retry unhealthy server: Server_st6 2025-12-17 23:43:07.710 INFO [SingleConnectionBrokerRequestHandler] [failure-detector-retry:172] Retrying unhealthy server: Server_st6 2025-12-17 23:43:07.710 INFO [MultiStageBrokerRequestHandler] [failure-detector-retry:172] Checking gRPC connection to unhealthy server: Server_st6 2025-12-17 23:43:07.710 INFO [QueryDispatcher] [failure-detector-retry:172] Still can't connect to server: Server_st6, current state: TRANSIENT_FAILURE # server included in routing after it comes up healthy 2025-12-17 23:46:37.483 INFO [BrokerRoutingManager] [ClusterChangeHandlingThread:225] Got excluded server: Server_st6 re-enabled, including it into the routing # failure detector confirms connection was successful minutes later 2025-12-17 23:48:17.711 INFO [MultiStageBrokerRequestHandler] [failure-detector-retry:172] Checking gRPC connection to unhealthy server: Server_st6 2025-12-17 23:48:17.711 INFO [QueryDispatcher] [failure-detector-retry:172] Successfully connected to server: Server_st6 2025-12-17 23:48:17.712 INFO [BaseExponentialBackoffRetryFailureDetector] [failure-detector-retry:172] Mark server: Server_st6 as healthy 2025-12-17 23:48:17.712 INFO [BrokerRoutingManager] [failure-detector-retry:172] Including server: Server_st6 to routing 2025-12-17 23:48:17.712 INFO [BrokerRoutingManager] [failure-detector-retry:172] Server: Server_st6 is not previously excluded, skipping updating the routing ``` I think the potential fix here is to remove the code in https://github.com/apache/pinot/blob/master/pinot-broker/src/main/java/org/apache/pinot/broker/routing/BrokerRoutingManager.java#L387-L390 and let the failure detector handle updating the routing map instead when it confirms the connection is healthy. I'll test this out tomorrow. -- 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]
