Skip to content

SOLR-18142: Fix CloudSolrClient cache state refresh; regression.#4176

Open
dsmiley wants to merge 1 commit intoapache:mainfrom
dsmiley:SOLR-18142
Open

SOLR-18142: Fix CloudSolrClient cache state refresh; regression.#4176
dsmiley wants to merge 1 commit intoapache:mainfrom
dsmiley:SOLR-18142

Conversation

@dsmiley
Copy link
Contributor

@dsmiley dsmiley commented Mar 2, 2026

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.testStaleStateRetryWaitsAfterSkipFailure

The 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 compute so that we can differentiate a Future that's already done/completed.

Recent improvements induced a new race condition caught by a flaky test.  State sometimes isn't refreshed.
Copy link
Contributor Author

@dsmiley dsmiley left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 =
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not critical but it's needless that this was non-final.

Comment on lines -1661 to -1665
if (closed) {
ExpiringCachedDocCollection cacheEntry = collectionStateCache.peek(collection);
DocCollection cached = cacheEntry == null ? null : cacheEntry.cached;
return CompletableFuture.completedFuture(cached);
}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Comment on lines +1662 to +1666
(key, existingFuture) -> {
// A refresh is still in progress; return it.
if (existingFuture != null && !existingFuture.isDone()) {
return existingFuture;
}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the essence of the fix. Everything else in the PR is an improvement but non-critical.

Comment on lines -1691 to -1695
future.whenCompleteAsync(
(result, error) -> {
collectionRefreshes.remove(key, future);
});
return future;
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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);
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it should always remove the same future, by the way.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant