[DRAFT] Fix pool renewal race#835
[DRAFT] Fix pool renewal race#835andrzej-jackowski-scylladb wants to merge 2 commits intoscylladb:masterfrom
Conversation
When pool creation races for the same host, a slower attempt can overwrite a pool that another thread already published and close connections with in-flight requests. Capture the previous pool before connection setup, then compare that state under the session lock before publishing the new pool. If another thread changed the pool, discard the stale pool instead of replacing the current one. Keep pool removals behind the same lock so the check observes all writers. Fixes: scylladb#317
Add a deterministic unit test for the case where another thread publishes a pool while a slower add attempt is still constructing its pool. This guards against closing in-flight connections by replacing the pool that should remain current. Refs: scylladb#317
|
@dkropachev, please reassign the review if there are more appropriate developers on the driver team. |
|
I'll look into it tomorrow. I did think about how to solve the issue, and couldn't figure out anything sensible, so I'm looking forward to reading your solution. |
|
Eh. Will it solve this specific issue? Maybe, I'm not sure. It is called in a few places:
The problem we are encountering is with cluster restart - adding nodes and session init are most likely not important here. What looks weird to me is the ending of that function: finally:
with host.lock:
host._currently_handling_node_up = False
# see if there are any pools to add or remove now that the host is marked up
for session in tuple(self.sessions):
session.update_created_pools()It first unsets the flag (allowing other However this seems not very likely to me now. if not pool or pool.is_shutdown:
# we don't eagerly set is_up on previously ignored hosts. None is included here
# to allow us to attempt connections to hosts that have gone from ignored to something
# else.
if distance != HostDistance.IGNORED and host.is_up in (True, None):
future = self.add_or_renew_pool(host, False)What is more likely here is actually the @run_in_executor
def on_down_potentially_blocking(self, host, is_host_addition):
self.profile_manager.on_down(host)
self.control_connection.on_down(host)
for session in tuple(self.sessions):
session.on_down(host)
for listener in self.listeners:
listener.on_down(host)
self._start_reconnector(host, is_host_addition)
def on_down(self, host, is_host_addition, expect_host_to_be_down=False):
"""
Intended for internal use only.
"""
if self.is_shutdown:
return
with host.lock:
was_up = host.is_up
# ignore down signals if we have open pools to the host
# this is to avoid closing pools when a control connection host became isolated
if self._discount_down_events and self.profile_manager.distance(host) != HostDistance.IGNORED:
connected = False
for session in tuple(self.sessions):
pool_states = session.get_pool_state()
pool_state = pool_states.get(host)
if pool_state:
connected |= pool_state['open_count'] > 0
if connected:
return
host.set_down()
if (not was_up and not expect_host_to_be_down) or host.is_currently_reconnecting():
return
log.warning("Host %s has been marked down", host)
self.on_down_potentially_blocking(host, is_host_addition)
I thought we could just copy the checks from |
nyh
left a comment
There was a problem hiding this comment.
I like this "optimistic locking" solution, where you allow creating the unneeded pool but end up not using it. Since this race very rarely happen, the small performance hit is not interesting.
I let Claude write a fix for this bug and it came up with something similar, but using a more elaborate mechanism where if we're already in the process of creating a pool, we return the existing future of that creation instead of creating a new one and later throwing it away. But the implementation is longer and in my opinion - not worth it.
|
Why is this fix in draft? What's missing? I think it's very important for our CI stability, we had a lot of tests fail on this problem over the years, and had to add workarounds for each. |
@Lorak-mmk here is an alternative fix which avoids creating the extra useless connections in the first place, and also explains why we have those multiple call to add_or_renew_pool in the first place: #838 That alternative fix was 100% written by AI, including the fix, the tests and the commit messages nicely explaining everything including how and why this race even happens. |
@dkropachev, @sylwiaszunejko, #317 continuously causes ScyllaDB CI failures. This is a draft PR that attempts to solve the problem, but I don't have a deep understanding of all the python-driver corner cases. Do you think this approach is worth pursuing?
=====
When pool creation races for the same host, a slower attempt can
overwrite a pool that another thread already published and close
connections with in-flight requests.
Capture the previous pool before connection setup, then compare
that state under the session lock before publishing the new pool.
If another thread changed the pool, discard the stale pool instead
of replacing the current one.
Keep pool removals behind the same lock so the check observes all
writers.
Fixes: #317
Pre-review checklist
./docs/source/.Fixes:annotations to PR description.