[multistage] Handling Server Restarts Gracefully
Note: Creating this ticket for tracking. Some existing solutions might already suffice, but I want to ideally find a path forward where we don't have to turn any config or tune any knob to get resilience from basic server operations like restarts.
While benchmarking MSE Lite Mode's resilience to server restarts, I found that even with Lite Mode, where only the leaf stages are run in the servers and everything else is run in the brokers, we were seeing high query failure rate during restart. The load used to test this was around 50 QPS with query latencies of ~50ms.
You can see the attached graph below. The first wave of queries is with the regular MSE, and the second wave of queries is using the Lite Mode. While Lite Mode has a much lower failure rate than regular MSE, the failure rate is still quite high, and ideally we want to aim for zero failures due to server restarts.
This experiment was run without any failure detector configured. See the section at the bottom.
Errors Seen
All the errors I saw were connection refused errors coming from the QueryDispatcher.
Query processing exceptions: {200=QueryExecutionError: Error dispatching query: 1048745684000196581 to server: some-host@{port1,port2} org.apache.pinot.query.service.dispatch.QueryDispatcher.execute(QueryDispatcher.java:312) org.apache.pinot.query.service.dispatch.QueryDispatcher.submit(QueryDispatcher.java:212) org.apache.pinot.query.service.dispatch.QueryDispatcher.submitAndReduce(QueryDispatcher.java:149) org.apache.pinot.broker.requesthandler.MultiStageBrokerRequestHandler.query(MultiStageBrokerRequestHandler.java:427) UNAVAILABLE: io exception io.grpc.Status.asRuntimeException(Status.java:535) io.grpc.stub.ClientCalls$StreamObserverToCallListenerAdapter.onClose(ClientCalls.java:487) io.grpc.internal.ClientCallImpl.closeObserver(ClientCallImpl.java:563) io.grpc.internal.ClientCallImpl.access$300(ClientCallImpl.java:70) finishConnect(..) failed: Connection refused: some-host/111111:12345 finishConnect(..) failed: Connection refused io.grpc.netty.shaded.io.netty.channel.unix.Errors.newConnectException0(Errors.java:155) io.grpc.netty.shaded.io.netty.channel.unix.Errors.handleConnectErrno(Errors.java:128) io.grpc.netty.shaded.io.netty.channel.unix.Socket.finishConnect(Socket.java:321) io.grpc.netty.shaded.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.doFinishConnect(AbstractEpollChannel.java:710) finishConnect(..) failed: Connection refused io.grpc.netty.shaded.io.netty.channel.unix.Errors.newConnectException0(Errors.java:155) io.grpc.netty.shaded.io.netty.channel.unix.Errors.handleConnectErrno(Errors.java:128) io.grpc.netty.shaded.io.netty.channel.unix.Socket.finishConnect(Socket.java:321) io.grpc.netty.shaded.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.doFinishConnect(AbstractEpollChannel.java:710) }
On Failure Detector
Recent work by @yashmayya to add Failure Detector is specifically to tackle this scenario (I suppose), where exceptions from failed query dispatch are caught and fed into the routing manager logic, to exclude servers that are failing query dispatch temporarily with custom backoff and retry strategies.
We'll be retrying the experiment with the Connection Based failure detector soon, but given this feature seems crucial to support basic reliability guarantees from the MSE, I think we should make this the default as soon as possible.
Related
I had assumed that I wouldn't need anything like Failure Detector to get restart resilience. I'll look into the server start-up/shutdown process again to see if we have misconfigured something on our end.
@Jackie-Jiang : it seems like we are starting the query server after the Helix Manager is connected. Shouldn't we start the query servers before helixManager is connected? This likely explains the errors we are seeing in our experiments too.
In our simple experiment even SSE queries (non multistage) are failing with with broker_request_send_error.
https://github.com/apache/pinot/blob/fb2e4dff467fa19e5eec1deb53380c3ad4183699/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/BaseServerStarter.java#L777-L780
Are your queries failing during the shutdown or the startup portion of the lifecycle?
A cursory look suggests both, but I need to dig a bit deeper.
We will be trying out a change next week and get back.
@ankitsultana Before server marking itself up (i.e. _helixAdmin.setConfig(_instanceConfigScope, Collections.singletonMap(Helix.IS_SHUTDOWN_IN_PROGRESS, Boolean.toString(false)));), broker shouldn't route any query to the server. Do you see server being queried before it marks itself up?
I just filed #16565 that may be related if you're doing externally orchestrated rolling restarts the way we are. It's not isolated to the v2 engine, though.
Hi @ankitsultana , I am interesting in troubleshooting this issue
@jadami10 : yeah we are seeing it for both v1 and v2 engine. It's become quite prominent so we'll try to prioritize this. @wirybeaver thanks for taking this up.
We were able to repro this for MSE by doing a kill -9 on the process. Once the server comes up, we see queries fail for 2-3 minutes. We did not see it on a graceful termination though.
We do have the server failure detector enabled on our brokers with:
pinot.broker.failure.detector.type: CONNECTION
Timeline:
2025-12-17 23:43:00 - `kill -9` command issued to a server
2025-12-17 23:46:37 - server completes startup + query errors start
2025-12-17 23:48:32 - query errors stop
Server startup log:
2025-12-17 23:46:37.367 INFO [ServerInstance] [Start a Pinot [SERVER]:156] Starting query server
2025-12-17 23:46:37.367 INFO [ServerInstance] [Start a Pinot [SERVER]:156] Starting query executor
2025-12-17 23:46:37.367 INFO [ServerQueryExecutorV1Impl] [Start a Pinot [SERVER]:156] Query executor started
2025-12-17 23:46:37.367 INFO [ServerInstance] [Start a Pinot [SERVER]:156] Starting query scheduler
2025-12-17 23:46:37.367 INFO [ServerInstance] [Start a Pinot [SERVER]:156] Starting Netty query server
2025-12-17 23:46:37.408 INFO [ServerInstance] [Start a Pinot [SERVER]:156] Starting worker query server
2025-12-17 23:46:37.408 INFO [QueryServer] [Start a Pinot [SERVER]:156] Starting QueryServer
2025-12-17 23:46:37.425 INFO [QueryServer] [Start a Pinot [SERVER]:156] Initialized QueryServer on port: 8421
2025-12-17 23:46:37.425 INFO [MailboxService] [Start a Pinot [SERVER]:156] Starting GrpcMailboxServer
2025-12-17 23:46:37.443 INFO [ServerQueryExecutorV1Impl] [Start a Pinot [SERVER]:156] Query executor started
2025-12-17 23:46:37.443 INFO [ServerInstance] [Start a Pinot [SERVER]:156] Finish starting query server
2025-12-17 23:46:37.443 INFO [ZKHelixAdmin] [Start a Pinot [SERVER]:156] Set configs with keys
2025-12-17 23:46:37.467 INFO [BaseServerStarter] [Start a Pinot [SERVER]:156] Pinot server ready
2025-12-17 23:46:37.472 INFO [PinotServiceManager] [Start a Pinot [SERVER]:156] Pinot Server instance [Server_st6] is Started...
2025-12-17 23:46:37.472 INFO [StartServiceManagerCommand] [Start a Pinot [SERVER]:156] Started Pinot [SERVER] instance [Server_st6] at 185.296s since launch
Broker log - first query error:
2025-12-17 23:46:37.840
ERROR [PinotClientRequest] [jersey-server-managed-async-executor-107801:137832] Query processing exceptions: {200=QueryExecutionError: Error dispatching query: 1972319055000858327 to server: pinotstreaming--0f98@{8421,8442}
[2025-12-17 23:46:37.840520] org.apache.pinot.query.service.dispatch.QueryDispatcher.processResults(QueryDispatcher.java:379)
[2025-12-17 23:46:37.840529] org.apache.pinot.query.service.dispatch.QueryDispatcher.execute(QueryDispatcher.java:334)
[2025-12-17 23:46:37.840535] org.apache.pinot.query.service.dispatch.QueryDispatcher.submit(QueryDispatcher.java:268)
[2025-12-17 23:46:37.840558] org.apache.pinot.query.service.dispatch.QueryDispatcher.submitAndReduce(QueryDispatcher.java:170)
[2025-12-17 23:46:37.840567] UNAVAILABLE: io exception
[2025-12-17 23:46:37.840572] io.grpc.Status.asRuntimeException(Status.java:532)
[2025-12-17 23:46:37.840579] io.grpc.stub.ClientCalls$StreamObserverToCallListenerAdapter.onClose(ClientCalls.java:564)
[2025-12-17 23:46:37.840586] io.grpc.internal.ClientCallImpl.closeObserver(ClientCallImpl.java:565)
[2025-12-17 23:46:37.840592] io.grpc.internal.ClientCallImpl.access$100(ClientCallImpl.java:72)
[2025-12-17 23:46:37.840599] Connection refused: pinotstreaming--0f98@8421
[2025-12-17 23:46:37.840604] Connection refused
[2025-12-17 23:46:37.840610] java.base/sun.nio.ch.Net.pollConnect(Native Method)
[2025-12-17 23:46:37.840615] java.base/sun.nio.ch.Net.pollConnectNow(Net.java:682)
[2025-12-17 23:46:37.840633] java.base/sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:973)
[2025-12-17 23:46:37.840645] io.grpc.netty.shaded.io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:336)
[2025-12-17 23:46:37.840649] Connection refused
[2025-12-17 23:46:37.840655] java.base/sun.nio.ch.Net.pollConnect(Native Method)
[2025-12-17 23:46:37.840661] java.base/sun.nio.ch.Net.pollConnectNow(Net.java:682)
[2025-12-17 23:46:37.840667] java.base/sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:973)
[2025-12-17 23:46:37.840676] io.grpc.netty.shaded.io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:336)
[2025-12-17 23:46:37.840679] }
@ankitsultana Could you please verify if you encountered the same exception as @dang-stripe shared? If so, the issue might be from the broker side, where it is not able to connect to the new restarted server even after server is fully ready for serving queries. cc @gortiz @yashmayya
I've done some investigation and think the failures we saw are specific to the failure detector + MSE + GRPC reconnection behavior, so it might not be related to the issue @ankitsultana saw.
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.
@dang-stripe When a server is restarted and ready to serve query, broker should start routing queries to it immediately. IMO we should fix the gRPC handling so that it is able to re-try the connection immediately. Given it is a different issue than the original topic, let's create a new issue and copy the context to the new issue.
Sounds good @Jackie-Jiang. I created https://github.com/apache/pinot/issues/17465.