SOLR-18142: Fix CloudSolrClient cache state refresh; regression.#4176
SOLR-18142: Fix CloudSolrClient cache state refresh; regression.#4176dsmiley wants to merge 1 commit intoapache:mainfrom
Conversation
Recent improvements induced a new race condition caught by a flaky test. State sometimes isn't refreshed.
dsmiley
left a comment
There was a problem hiding this comment.
The test I was looked at passed using the same reproduction technique.
I haven't checked on the rest of the tests, yet.
| private final RequestReplicaListTransformerGenerator requestRLTGenerator; | ||
| private final boolean parallelUpdates; | ||
| private ExecutorService threadPool = | ||
| private final ExecutorService threadPool = |
There was a problem hiding this comment.
not critical but it's needless that this was non-final.
| if (closed) { | ||
| ExpiringCachedDocCollection cacheEntry = collectionStateCache.peek(collection); | ||
| DocCollection cached = cacheEntry == null ? null : cacheEntry.cached; | ||
| return CompletableFuture.completedFuture(cached); | ||
| } |
There was a problem hiding this comment.
I found it confusing for this method to have two code paths for the closed scenario. So I centralized it to one spot within the map.compute call.
| (key, existingFuture) -> { | ||
| // A refresh is still in progress; return it. | ||
| if (existingFuture != null && !existingFuture.isDone()) { | ||
| return existingFuture; | ||
| } |
There was a problem hiding this comment.
This is the essence of the fix. Everything else in the PR is an improvement but non-critical.
| future.whenCompleteAsync( | ||
| (result, error) -> { | ||
| collectionRefreshes.remove(key, future); | ||
| }); | ||
| return future; |
There was a problem hiding this comment.
It's much lighter weight & simpler to read to incorporate this into the single lambda callback to occur after the collection is loaded.
I spent a fair amount of time previously trying to assure myself on the nuances of whenComplete vs whenCompleteAsync, and on returning the result of this future vs not, or having the outer method actually do this. Played with a debugger to inspect threads; putting sleep in places and running tests. It was educational but I concluded we can do something much simpler.
| } finally { | ||
| stateRefreshSemaphore.release(); | ||
| // Remove the entry in case of many collections | ||
| collectionRefreshes.remove(key); |
There was a problem hiding this comment.
it should always remove the same future, by the way.
Recent improvements induced a new race condition caught by a flaky test. State sometimes isn't refreshed.
https://issues.apache.org/jira/browse/SOLR-18142
See
CloudSolrClientCacheTest.testStaleStateRetryWaitsAfterSkipFailureThe Race Condition:
- Thread 1: getDocCollection() triggers refresh → creates CompletableFuture → calls ref.get() (refGets=1) → future completes
- Thread 2 (async): Future completion callback runs to remove it from collectionRefreshes map
- Race Window: Between future completing and callback removing it from map
- Thread 1: Detects INVALID_STATE error → calls triggerCollectionRefresh() again
- If callback hasn't run yet, computeIfAbsent finds the completed future → returns it without calling ref.get() again
- Result: refGets stays at 1 instead of 2 → test fails
Solution is to use
computeso that we can differentiate a Future that's already done/completed.