Copilot commented on code in PR #4312:
URL: https://github.com/apache/solr/pull/4312#discussion_r3118277535
##########
solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java:
##########
@@ -1313,6 +1308,33 @@ protected NamedList<Object> requestWithRetryOnStaleState(
MAX_STALE_RETRIES,
wasCommError,
errorCode);
+ // For 503 "no leader" errors (not plain comm errors where the node
is fully dead),
+ // wait for the cluster state to refresh from ZooKeeper before
retrying.
+ // Without this wait, all MAX_STALE_RETRIES retries fire within
milliseconds of each
+ // other using the same stale routes, hitting the same dead leader
repeatedly and
+ // exhausting all retries before leader election can complete.
+ // We use a bounded wait (10 s) so a stuck ZK / stalled election
cannot block
+ // the caller thread indefinitely; on timeout we log and continue
with the retry
+ // using whatever state is currently available.
+ if (!wasCommError && requestedCollections != null &&
!requestedCollections.isEmpty()) {
+ for (DocCollection ext : requestedCollections) {
+ try {
+ triggerCollectionRefresh(ext.getName()).get(10,
TimeUnit.SECONDS);
+ } catch (TimeoutException te) {
+ log.warn(
+ "Timed out waiting for cluster state refresh for
collection {} before retry; "
+ + "proceeding with retry using current state",
+ ext.getName());
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
Review Comment:
On InterruptedException during the refresh wait you re-interrupt the thread
but then continue into the recursive retry. This effectively ignores
cancellation and can lead to surprising behavior (and is inconsistent with
waitForCollectionRefresh(), which throws on interrupt). Consider aborting
retries and propagating an exception once interrupted, or returning/throwing
after restoring the interrupt flag.
```suggestion
Thread.currentThread().interrupt();
throw new SolrServerException(
"Interrupted while waiting for cluster state refresh
before retry for collection "
+ ext.getName(),
ie);
```
##########
solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java:
##########
@@ -1313,6 +1308,33 @@ protected NamedList<Object> requestWithRetryOnStaleState(
MAX_STALE_RETRIES,
wasCommError,
errorCode);
+ // For 503 "no leader" errors (not plain comm errors where the node
is fully dead),
+ // wait for the cluster state to refresh from ZooKeeper before
retrying.
+ // Without this wait, all MAX_STALE_RETRIES retries fire within
milliseconds of each
+ // other using the same stale routes, hitting the same dead leader
repeatedly and
+ // exhausting all retries before leader election can complete.
+ // We use a bounded wait (10 s) so a stuck ZK / stalled election
cannot block
+ // the caller thread indefinitely; on timeout we log and continue
with the retry
+ // using whatever state is currently available.
+ if (!wasCommError && requestedCollections != null &&
!requestedCollections.isEmpty()) {
+ for (DocCollection ext : requestedCollections) {
+ try {
+ triggerCollectionRefresh(ext.getName()).get(10,
TimeUnit.SECONDS);
+ } catch (TimeoutException te) {
+ log.warn(
+ "Timed out waiting for cluster state refresh for
collection {} before retry; "
+ + "proceeding with retry using current state",
+ ext.getName());
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ } catch (ExecutionException ee) {
+ log.warn(
+ "Error refreshing cluster state for collection {} before
retry",
+ ext.getName(),
+ ee.getCause());
+ }
+ }
Review Comment:
The 503 retry path blocks up to 10s per collection *per retry* and does the
waits sequentially. If an alias resolves to multiple collections, this can
multiply into a long stall (N collections × 10s × MAX_STALE_RETRIES) on a
single caller thread. Consider using a single overall deadline (or
CompletableFuture.allOf over the refresh futures) and waiting in parallel with
remaining-time budgeting so the total wait is bounded.
```suggestion
// We use a bounded wait (10 s total) so a stuck ZK / stalled
election cannot block
// the caller thread indefinitely; on timeout we log and continue
with the retry
// using whatever state is currently available.
if (!wasCommError && requestedCollections != null &&
!requestedCollections.isEmpty()) {
List<CompletableFuture<?>> refreshFutures = new
ArrayList<>(requestedCollections.size());
for (DocCollection ext : requestedCollections) {
final String collectionName = ext.getName();
refreshFutures.add(
triggerCollectionRefresh(collectionName)
.handle(
(ignored, throwable) -> {
if (throwable != null) {
Throwable cause =
throwable.getCause() != null ?
throwable.getCause() : throwable;
log.warn(
"Error refreshing cluster state for
collection {} before retry",
collectionName,
cause);
}
return null;
}));
}
try {
CompletableFuture
.allOf(refreshFutures.toArray(new CompletableFuture<?>[0]))
.get(10, TimeUnit.SECONDS);
} catch (TimeoutException te) {
log.warn(
"Timed out waiting for cluster state refresh for
collections {} before retry; "
+ "proceeding with retry using current state",
requestedCollections.stream()
.map(DocCollection::getName)
.collect(Collectors.toList()));
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
}
```
##########
solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java:
##########
@@ -1313,6 +1308,33 @@ protected NamedList<Object> requestWithRetryOnStaleState(
MAX_STALE_RETRIES,
wasCommError,
errorCode);
+ // For 503 "no leader" errors (not plain comm errors where the node
is fully dead),
+ // wait for the cluster state to refresh from ZooKeeper before
retrying.
+ // Without this wait, all MAX_STALE_RETRIES retries fire within
milliseconds of each
+ // other using the same stale routes, hitting the same dead leader
repeatedly and
+ // exhausting all retries before leader election can complete.
+ // We use a bounded wait (10 s) so a stuck ZK / stalled election
cannot block
+ // the caller thread indefinitely; on timeout we log and continue
with the retry
Review Comment:
There are existing SolrJ retry-related tests (e.g.,
solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientRetryTest.java),
but this new 503-specific refresh-wait behavior isn’t covered. Adding a
focused test that simulates a RouteException(503) and asserts that a collection
refresh is triggered and awaited (or that retries are delayed until state
changes) would help prevent regressions and confirm the intended behavior.
##########
solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java:
##########
@@ -1285,15 +1285,10 @@ protected NamedList<Object>
requestWithRetryOnStaleState(
String name = ext.getName();
ExpiringCachedDocCollection cacheEntry =
collectionStateCache.peek(name);
if (cacheEntry != null) {
- if (wasCommError) {
- cacheEntry.maybeStale = true;
- } else {
- boolean markedStale =
-
cacheEntry.markMaybeStaleIfOutsideBackoff(retryExpiryTimeNano);
- if (markedStale && cacheEntry.shouldRetry()) {
- triggerCollectionRefresh(name);
- }
- }
+ // For both comm errors and 503 (no leader), mark state as stale
immediately.
+ // For 503 we bypass the backoff since we will wait for the
refresh below
+ // before retrying, which naturally throttles the retry rate.
+ cacheEntry.maybeStale = true;
Review Comment:
This change removes the existing retryExpiryTimeNano backoff for 503 errors
by unconditionally setting maybeStale=true (instead of using
markMaybeStaleIfOutsideBackoff). That means a burst of 503s can now trigger
much more frequent state refreshes (and refresh-waits) across many concurrent
client threads. If the goal is specifically to handle leader-election windows,
consider keeping some throttling/jitter (or only bypassing backoff for the
first 503 in a retry chain) to avoid ZK/cluster-state refresh storms under
sustained 503 conditions.
```suggestion
// Throttle stale marking so repeated comm errors or 503s do
not cause
// excessive concurrent cluster-state refresh attempts.
markMaybeStaleIfOutsideBackoff(cacheEntry);
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]