Pierre Salagnac created SOLR-17405: -------------------------------------- Summary: Zookeeper session can be re-established by multiple threads concurrently Key: SOLR-17405 URL: https://issues.apache.org/jira/browse/SOLR-17405 Project: Solr Issue Type: Bug Security Level: Public (Default Security Level. Issues are Public) Affects Versions: 9.6, 8.11 Reporter: Pierre Salagnac Attachments: stack.png
Because of a bug in SolrCloud, the Zookeeper session can be re-established by multiple threads concurrently when an expiration occurs. This portion of the code assumes it is mono-threaded. Because of the bug, the last thread re-establishing the session can waif for 30 seconds per core, waiting for it to be marked {{DOWN}} while it was previously marked {{ACTIVE}} by another thread. With a high number of cores, the Solr server can hang for hours before taking traffic again. Following exception shows two threads were reestablishing the session concurrently. {{ZkController.createEphemeralLiveNode()}} should never be invoked twice for the same Zookeeper session. !stack.png! h2. Root cause This bug occurs because several threads can re-establish the session concurrently. It cannot happen at the first expiration of the session, thanks to a thread pool with a single thread to execute the zookeeper Watcher. Bellow is a code snippet from class {{SolrZkClient.ProcessWatchWithExecutor}} {code:java} if (watcher instanceof ConnectionManager) { zkConnManagerCallbackExecutor.submit(() -> watcher.process(event)); } else { ....... } {code} Using this dedicated thread pool (with a single thread) is supposed to ensure we don’t handle watches for connection related events with multiple threads. This works well for the first session expiration. Now, when we re-establish the session after the first expiration, we don’t use this wrapper to register the watch. It is done directly in {{ConnectionManager}} without wrapping the ZK watch. In the following snippet, _“this”_ is the ZK watcher instance, but it is not wrapper to use a {{{}ProcessWatchWithExecutor{}}}. This means the next events will directly be handled by any ZK callback thread. {code:java} connectionStrategy.reconnect(zkServerAddress,client.getZkClientTimeout(), this, {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@solr.apache.org For additional commands, e-mail: issues-h...@solr.apache.org