You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2022/06/17 01:53:41 UTC

[GitHub] [solr] patsonluk opened a new pull request, #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

patsonluk opened a new pull request, #909:
URL: https://github.com/apache/solr/pull/909

   https://issues.apache.org/jira/browse/SOLR-16257
   
   # Description
   
   As described in the Jira issue, `ZkStateReader` fields `collectionWatches` and `watchedCollectionStates` can run into race conditions. 
   
   And such condition might produce stale `clusterState` for some collection which can no longer be updated (until a new watch is registered for such collection)
   
   The existing design is intended to synchronize the 2 map fields `watchedCollectionStates` and `collectionWatches` (ie if a collection is no longer watched in `collectionWatches`, then no entry for that collection should exist in `watchedCollectionStates`). However, it is hard to guarantee such "synchronization", one way to break it is demonstrated in "Steps to reproduce a race condition" within the Jira issue
   
   # Solution
   
   Perhaps it's easier to simply eliminate  `watchedCollectionStates` and add such `state` (as `DocCollection`) to the `CollectionWatch` itself (now added a new class `DocCollectionWatch` with contains a ref to the `DocCollection`) , such that we no longer need to worry about entry removed in  `collectionWatches` somehow still remains in `watchedCollectionStates`
   
   # Tests
   
   No extra test cases yet. Will update.
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/solr/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [ ] I have given Solr maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [ ] I have run `./gradlew check`.
   - [ ] I have added tests for my changes.
   - [ ] I have added documentation for the [Reference Guide](https://github.com/apache/solr/tree/main/solr/solr-ref-guide)
   


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922387781


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -539,17 +672,17 @@ private void constructState(Set<String> changedCollections) {
     if (log.isDebugEnabled()) {
       log.debug(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
-          collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
+          collectionWatches.watchedCollections().size(),
+          collectionWatches.activeCollectionCount(),
           lazyCollectionStates.keySet().size(),
           clusterState.getCollectionStates().size());
     }
 
     if (log.isTraceEnabled()) {
       log.trace(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
-          collectionWatches.keySet(),
-          watchedCollectionStates.keySet(),
+          collectionWatches.watchedCollections(),
+          collectionWatches.activeCollections(),
           lazyCollectionStates.keySet(),
           clusterState.getCollectionStates());
     }

Review Comment:
   I believe it could change but it should be okay.
   
   As far as I understand, we only need a strong guarantee/synchronization on the write ops. As for read op, it appears none of the read operation requires such guarantee, ie even if the value read has changed or is inconsistent with the `watchedCollectionStates`, it should not cause any issues. 
   
   The core of this PR is to remove the race conditions between write operations of 2 separate maps, which can cause stale state. And now since all write ops are on the same map with sync, we should be safe 😊  . As for read ops, it's pretty much the same as before, which does not need to have such strong guarantee I suppose? 
   



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922392959


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -525,8 +652,14 @@ private void constructState(Set<String> changedCollections) {
     Map<String, ClusterState.CollectionRef> result = new LinkedHashMap<>();
 
     // Add collections
-    for (Map.Entry<String, DocCollection> entry : watchedCollectionStates.entrySet()) {
-      result.put(entry.getKey(), new ClusterState.CollectionRef(entry.getValue()));
+    for (Entry<String, StatefulCollectionWatch> entry :
+        collectionWatches.watchedCollectionEntries()) {
+      if (entry.getValue().currentState != null) {
+        // if the doc is null for the collection watch, then it should not be inserted into the
+        // state
+        result.putIfAbsent(

Review Comment:
   I'm scratch my head too 🤣 , as the latest code is not doing `putIfAbsent` here either. I feel like I could be just copying the original code from some earlier [commit](https://github.com/apache/solr/blame/54c5dd7d6d8b486df9132e637a6d7c5871ac4664/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java#L591)  
   
   Anyway, using `put` instead!



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r921503461


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +245,138 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch =
+          statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>>
+        watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      StatefulCollectionWatch finalWatch =
+          statefulWatchesByCollectionName.computeIfPresent(
+              collection,
+              (col, watch) -> {
+                DocCollection oldState = watch.currentState;
+                if (oldState == null && newState == null) {
+                  // OK, the collection not yet exist in ZK or already deleted
+                } else if (oldState == null) {
+                  if (log.isDebugEnabled()) {
+                    log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+                  }
+                  watch.currentState = newState;
+                } else if (newState == null) {
+                  log.debug("Removing cached collection state for [{}]", collection);
+                  watch.currentState = null;
+                } else { // both new and old states are non-null
+                  int oldCVersion =
+                      oldState.getPerReplicaStates() == null
+                          ? -1
+                          : oldState.getPerReplicaStates().cversion;
+                  int newCVersion =
+                      newState.getPerReplicaStates() == null
+                          ? -1
+                          : newState.getPerReplicaStates().cversion;
+                  if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                      || oldCVersion < newCVersion) {
+                    watch.currentState = newState;
+                    if (log.isDebugEnabled()) {
+                      log.debug(
+                          "Updating data for [{}] from [{}] to [{}]",
+                          collection,
+                          oldState.getZNodeVersion(),
+                          newState.getZNodeVersion());
+                    }
+                  }
+                }
+                return watch;
+              });
+      return finalWatch != null;
+    }

Review Comment:
   As I mentioned in https://github.com/apache/solr/pull/909#discussion_r921419678, I don't think that comment is correct, so we should be good to go with the new behavior.
   
   > 1.  If both new and old states are null (which could happen if either the collection is not yet created or is removed), it would still return true
   
   It's confusing but I don't think that is the case. Looking at the relevant part of the old implementation:
   
   ```java
    if (oldState == null) {
           if (watchedCollectionStates.putIfAbsent(coll, newState) == null) {
             if (log.isDebugEnabled()) {
               log.debug("Add data for [{}] ver [{}]", coll, newState.getZNodeVersion());
             }
             updated = true;
             break;
           }
         }
   ```
   
   [putIfAbsent()](https://docs.oracle.com/javase/8/docs/api/java/util/Map.html#putIfAbsent-K-V-) returns the previous value associated with that key, so `null` means that there was no previous value and the `putIfAbsent()` successfully executed. It doesn't actually check what the `newState` is.
   
   So it would return `true` if the old state was null, no matter what. We probably don't want to keep that functionality. No need to update collections if a state went from `null` -> `null`.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915236042


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   Thank you for the detailed explanations! @HoustonPutman 😊 
   
   I have refactored the code but somehow after the refactoring the test case `ZkStateReaderTest#testWatchRaceCondition` started to fail consistently almost every time , while I also ran it on the pre-refactor code around 10 times and it didn't fail once.
   
   I am still trying to figure out why, but meanwhile this is my change, perhaps someone caught ur 👀 :
   https://github.com/fullstorydev/solr/compare/patsonluk/SOLR-16257/race-condition-can-lead-to-inconsistent-state...fullstorydev:solr:patsonluk/SOLR-16257/refactored?expand=1



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915422007


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -540,7 +627,7 @@ private void constructState(Set<String> changedCollections) {
       log.debug(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
           collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
+          collectionWatches.activeCollections().size(),

Review Comment:
   You already have a method that gets all the names (activeCollections). Having two methods for different things is fine, even if one can be implemented in terms of the other because we are able to make it so much cleaner for the more common use case.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r916144715


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentDoc : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, DocCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentDoc != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link DocCollectionWatch} if the
+     * collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      if (watch != null) {
+        DocCollection oldState = watch.currentDoc;
+        if (oldState == null && newState == null) {
+          // OK, the collection not yet exist in ZK
+        } else if (oldState == null) {
+          if (log.isDebugEnabled()) {
+            log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+          }
+          watch.currentDoc = newState;
+        } else if (newState == null) {
+          log.debug("Removing cached collection state for [{}]", collection);
+          watch.currentDoc = null;
+        } else { // both new and old states are non-null
+          int oldCVersion =
+              oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
+          int newCVersion =
+              newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
+          if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+              || oldCVersion < newCVersion) {
+            watch.currentDoc = newState;
+            if (log.isDebugEnabled()) {
+              log.debug(
+                  "Updating data for [{}] from [{}] to [{}]",
+                  collection,
+                  oldState.getZNodeVersion(),
+                  newState.getZNodeVersion());
+            }
+          }
+        }
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  private static class DocCollectionWatch<T> extends CollectionWatch<T> {

Review Comment:
   Agreed! especially that it has a `private DocCollection currentState;` which is pretty specific and not generic anyway



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
risdenk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1208139621

   FYI this has been causing NPE in ZkStateReaderTest - https://github.com/apache/solr/pull/966


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r910100174


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentDoc : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, DocCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentDoc != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link DocCollectionWatch} if the
+     * collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      if (watch != null) {
+        DocCollection oldState = watch.currentDoc;
+        if (oldState == null && newState == null) {
+          // OK, the collection not yet exist in ZK
+        } else if (oldState == null) {
+          if (log.isDebugEnabled()) {
+            log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+          }
+          watch.currentDoc = newState;
+        } else if (newState == null) {
+          log.debug("Removing cached collection state for [{}]", collection);
+          watch.currentDoc = null;
+        } else { // both new and old states are non-null
+          int oldCVersion =
+              oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
+          int newCVersion =
+              newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
+          if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+              || oldCVersion < newCVersion) {
+            watch.currentDoc = newState;
+            if (log.isDebugEnabled()) {
+              log.debug(
+                  "Updating data for [{}] from [{}] to [{}]",
+                  collection,
+                  oldState.getZNodeVersion(),
+                  newState.getZNodeVersion());
+            }
+          }
+        }
+        return true;
+      } else {
+        return false;
+      }
+    }

Review Comment:
   > Just a really minor nit, perhaps we can even remove the `if (watch != null)` check now as it's `computeIfPresent`?
   
   yeah, that's a good call.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1187743469

   @HoustonPutman Thanks for triggering the workflow (I assume it was you? 😅 ), Now that all the checks have passed, what are the next steps? (trying to read up on https://cwiki.apache.org/confluence/display/SOLR/Git+commit+process and https://cwiki.apache.org/confluence/display/solr/HowToContribute#HowToContribute-Contributingyourwork)
   
   There's no rush for this PR, but just want to make sure I am not holding off on the process of any missing items.
   
   Big thank you again @HoustonPutman and @madrob !


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922387781


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -539,17 +672,17 @@ private void constructState(Set<String> changedCollections) {
     if (log.isDebugEnabled()) {
       log.debug(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
-          collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
+          collectionWatches.watchedCollections().size(),
+          collectionWatches.activeCollectionCount(),
           lazyCollectionStates.keySet().size(),
           clusterState.getCollectionStates().size());
     }
 
     if (log.isTraceEnabled()) {
       log.trace(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
-          collectionWatches.keySet(),
-          watchedCollectionStates.keySet(),
+          collectionWatches.watchedCollections(),
+          collectionWatches.activeCollections(),
           lazyCollectionStates.keySet(),
           clusterState.getCollectionStates());
     }

Review Comment:
   I believe it could change but it should be okay.
   
   As far as I understand, we only need a strong guarantee/synchronization on the write ops. As for read op, it appears none of the read operation requires such guarantee, ie even if the value read has changed or is inconsistent with the `watchedCollectionStates`, it should not cause any serious issues. (it might print value that does not reflect the latest set or fetch/not fetch something - but these are not new behaviors) 
   
   The core of this PR is to remove the race conditions between write operations of 2 separate maps, which can cause stale state. And now since all write ops are on the same map with sync, we should be safe 😊  . As for read ops, it's pretty much the same as before, which does not need to have such strong guarantee I suppose? 
   



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r921637315


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +245,138 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch =
+          statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>>
+        watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      StatefulCollectionWatch finalWatch =
+          statefulWatchesByCollectionName.computeIfPresent(
+              collection,
+              (col, watch) -> {
+                DocCollection oldState = watch.currentState;
+                if (oldState == null && newState == null) {
+                  // OK, the collection not yet exist in ZK or already deleted
+                } else if (oldState == null) {
+                  if (log.isDebugEnabled()) {
+                    log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+                  }
+                  watch.currentState = newState;
+                } else if (newState == null) {
+                  log.debug("Removing cached collection state for [{}]", collection);
+                  watch.currentState = null;
+                } else { // both new and old states are non-null
+                  int oldCVersion =
+                      oldState.getPerReplicaStates() == null
+                          ? -1
+                          : oldState.getPerReplicaStates().cversion;
+                  int newCVersion =
+                      newState.getPerReplicaStates() == null
+                          ? -1
+                          : newState.getPerReplicaStates().cversion;
+                  if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                      || oldCVersion < newCVersion) {
+                    watch.currentState = newState;
+                    if (log.isDebugEnabled()) {
+                      log.debug(
+                          "Updating data for [{}] from [{}] to [{}]",
+                          collection,
+                          oldState.getZNodeVersion(),
+                          newState.getZNodeVersion());
+                    }
+                  }
+                }
+                return watch;
+              });
+      return finalWatch != null;
+    }

Review Comment:
   Cool thanks! I will accept the change. My original code was to preserve the old behavior of `updateWatchedCollection` to play on the safe side. But it appears that change of behavior is actually more "correct", so im all for it. thanks! 🙇🏼 



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r921636428


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,95 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, StatefulCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection

Review Comment:
   Thank you for the detailed explanations @HoustonPutman !
   
   So it appears that `constructState` serves at least 2 purposes:
   1. Construct the clusterstate again as there are changes in the watched collection states
   2. Notify all the watchers as defined in `changedCollections` param
   
   So i agree that for 1., it only makes sense to re-construct the state if indeed the doc collection has been changed
   
   As for 2, I don't understand enough to tell whether it matters, like whether `forciblyRefreshAllClusterStateSlow` would expect all watches be notified, whether there's any change on the corresponding collection or not. But i trust your analysis!
   
   
    



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922395415


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +246,134 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch = statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>> watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether the state has changed for the watched collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      AtomicBoolean stateHasChanged = new AtomicBoolean(false);
+      statefulWatchesByCollectionName.computeIfPresent(
+          collection,
+          (col, watch) -> {
+            DocCollection oldState = watch.currentState;
+            if (oldState == null && newState == null) {
+              // OK, the collection not yet exist in ZK or already deleted
+            } else if (oldState == null) {
+              if (log.isDebugEnabled()) {
+                log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+              }
+              watch.currentState = newState;
+            } else if (newState == null) {
+              log.debug("Removing cached collection state for [{}]", collection);
+              watch.currentState = null;
+            } else { // both new and old states are non-null
+              int oldCVersion =
+                  oldState.getPerReplicaStates() == null
+                      ? -1
+                      : oldState.getPerReplicaStates().cversion;
+              int newCVersion =
+                  newState.getPerReplicaStates() == null
+                      ? -1
+                      : newState.getPerReplicaStates().cversion;
+              if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                  || oldCVersion < newCVersion) {
+                watch.currentState = newState;
+                if (log.isDebugEnabled()) {
+                  log.debug(
+                      "Updating data for [{}] from [{}] to [{}]",
+                      collection,
+                      oldState.getZNodeVersion(),
+                      newState.getZNodeVersion());
+                }
+              }
+            }
+            stateHasChanged.set(oldState != watch.currentState);
+            return watch;
+          });
+
+      return stateHasChanged.get();
+    }
+
+    /**
+     * Computes the new StatefulCollectionWatch by the supplied remappingFunction.

Review Comment:
   Added!



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922378085


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -29,171 +38,387 @@
 import org.apache.solr.cloud.ZkTestServer;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocCollectionWatcher;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.CommonTestInjection;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.util.TimeOut;
+import org.junit.After;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZkStateReaderTest extends SolrTestCaseJ4 {
-
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final long TIMEOUT = 30;
 
-  public void testExternalCollectionWatchedNotWatched() throws Exception {
-    Path zkDir = createTempDir("testExternalCollectionWatchedNotWatched");
-    ZkTestServer server = new ZkTestServer(zkDir);
-    SolrZkClient zkClient = null;
-    ZkStateReader reader = null;
-
-    try {
-      server.run();
-
-      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
-      ZkController.createClusterZkNodes(zkClient);
-
-      reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-
-      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
-
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-
-      // create new collection
-      ZkWriteCommand c1 =
-          new ZkWriteCommand(
-              "c1",
-              new DocCollection(
-                  "c1",
-                  new HashMap<>(),
-                  Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
-                  DocRouter.DEFAULT,
-                  0));
-      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(c1), null);
-      writer.writePendingUpdates();
-      reader.forceUpdateCollection("c1");
-
-      assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
-      reader.registerCore("c1");
-      assertFalse(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
-      reader.unregisterCore("c1");
-      assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
+  private static class TestFixture implements Closeable {
+    private final ZkTestServer server;
+    private final SolrZkClient zkClient;
+    private final ZkStateReader reader;
+    private final ZkStateWriter writer;
+
+    private TestFixture(
+        ZkTestServer server, SolrZkClient zkClient, ZkStateReader reader, ZkStateWriter writer) {
+      this.server = server;
+      this.zkClient = zkClient;
+      this.reader = reader;
+      this.writer = writer;
+    }
 
-    } finally {
+    @Override
+    public void close() throws IOException {
       IOUtils.close(reader, zkClient);
-      server.shutdown();
+      try {
+        server.shutdown();
+      } catch (InterruptedException e) {
+        // ok. Shutting down anyway
+      }
     }
   }
 
-  public void testCollectionStateWatcherCaching() throws Exception {
-    Path zkDir = createTempDir("testCollectionStateWatcherCaching");
-
-    ZkTestServer server = new ZkTestServer(zkDir);
+  private TestFixture fixture = null;
 
-    SolrZkClient zkClient = null;
-    ZkStateReader reader = null;
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    fixture = setupTestFixture(getTestName());
+  }
 
-    try {
-      server.run();
-
-      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
-      ZkController.createClusterZkNodes(zkClient);
-
-      reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-
-      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
-      DocCollection state =
-          new DocCollection(
-              "c1",
-              new HashMap<>(),
-              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
-              DocRouter.DEFAULT,
-              0);
-      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
-      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
-      writer.writePendingUpdates();
-      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
-      reader.waitForState(
-          "c1", 1, TimeUnit.SECONDS, (liveNodes, collectionState) -> collectionState != null);
-
-      Map<String, Object> props = new HashMap<>();
-      props.put("x", "y");
-      props.put(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME);
-      state = new DocCollection("c1", new HashMap<>(), props, DocRouter.DEFAULT, 0);
-      wc = new ZkWriteCommand("c1", state);
-      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
-      writer.writePendingUpdates();
-
-      boolean found = false;
-      TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
-      while (!timeOut.hasTimedOut()) {
-        DocCollection c1 = reader.getClusterState().getCollection("c1");
-        if ("y".equals(c1.getStr("x"))) {
-          found = true;
-          break;
-        }
-      }
-      assertTrue("Could not find updated property in collection c1 even after 5 seconds", found);
-    } finally {
-      IOUtils.close(reader, zkClient);
-      server.shutdown();
+  @After
+  public void tearDown() throws Exception {
+    if (fixture != null) {
+      fixture.close();
     }
+    super.tearDown();
   }
 
-  public void testWatchedCollectionCreation() throws Exception {
-    Path zkDir = createTempDir("testWatchedCollectionCreation");
-
+  private static TestFixture setupTestFixture(String testPrefix) throws Exception {
+    Path zkDir = createTempDir(testPrefix);
     ZkTestServer server = new ZkTestServer(zkDir);
+    server.run();
+    SolrZkClient zkClient =
+        new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+    ZkController.createClusterZkNodes(zkClient);
 
-    SolrZkClient zkClient = null;
-    ZkStateReader reader = null;
-
-    try {
-      server.run();
+    ZkStateReader reader = new ZkStateReader(zkClient);
+    reader.createClusterStateWatchersAndUpdate();
 
-      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
-      ZkController.createClusterZkNodes(zkClient);
+    ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
 
-      reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-      reader.registerCore("c1");
+    return new TestFixture(server, zkClient, reader, writer);
+  }
 
-      // Initially there should be no c1 collection.
-      assertNull(reader.getClusterState().getCollectionRef("c1"));
+  public void testExternalCollectionWatchedNotWatched() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+    // create new collection
+    ZkWriteCommand c1 =
+        new ZkWriteCommand(
+            "c1",
+            new DocCollection(
+                "c1",
+                new HashMap<>(),
+                Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+                DocRouter.DEFAULT,
+                0));
+
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(c1), null);
+    writer.writePendingUpdates();
+    reader.forceUpdateCollection("c1");
+
+    assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
+    reader.registerCore("c1");
+    assertFalse(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
+    reader.unregisterCore("c1");
+    assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
+  }
 
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-      reader.forceUpdateCollection("c1");
+  public void testCollectionStateWatcherCaching() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+    DocCollection state =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+    assertTrue(fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+    reader.waitForState(
+        "c1", 1, TimeUnit.SECONDS, (liveNodes, collectionState) -> collectionState != null);
+
+    Map<String, Object> props = new HashMap<>();
+    props.put("x", "y");
+    props.put(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME);
+    state = new DocCollection("c1", new HashMap<>(), props, DocRouter.DEFAULT, 0);
+    wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+
+    boolean found = false;
+    TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    while (!timeOut.hasTimedOut()) {
+      DocCollection c1 = reader.getClusterState().getCollection("c1");
+      if ("y".equals(c1.getStr("x"))) {
+        found = true;
+        break;
+      }
+    }
+    assertTrue("Could not find updated property in collection c1 even after 5 seconds", found);
+  }
 
-      // Still no c1 collection, despite a collection path.
-      assertNull(reader.getClusterState().getCollectionRef("c1"));
+  public void testWatchedCollectionCreation() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+
+    reader.registerCore("c1");
+
+    // Initially there should be no c1 collection.
+    assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+    reader.forceUpdateCollection("c1");
+
+    // Still no c1 collection, despite a collection path.
+    assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+    // create new collection
+    DocCollection state =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+
+    assertTrue(fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+    // reader.forceUpdateCollection("c1");
+    reader.waitForState("c1", TIMEOUT, TimeUnit.SECONDS, (n, c) -> c != null);
+    ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+    assertNotNull(ref);
+    assertFalse(ref.isLazilyLoaded());
+  }
 
-      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+
+    reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+    reader.forciblyRefreshAllClusterStateSlow();
+    // Initially there should be no c1 collection.
+    assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+    // create new collection
+    DocCollection state =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+
+    assertTrue(fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+    reader.forciblyRefreshAllClusterStateSlow();
+    ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+    assertNotNull(ref);
+    assertFalse(ref.isLazilyLoaded());
+    assertEquals(0, ref.get().getZNodeVersion());
+
+    // update the collection
+    state =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            ref.get().getZNodeVersion());
+    wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+
+    reader.forciblyRefreshAllClusterStateSlow();
+    ref = reader.getClusterState().getCollectionRef("c1");
+    assertNotNull(ref);
+    assertFalse(ref.isLazilyLoaded());
+    assertEquals(1, ref.get().getZNodeVersion());
+
+    // delete the collection c1, add a collection c2 that is NOT watched
+    ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+    state =
+        new DocCollection(
+            "c2",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+    writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+    writer.writePendingUpdates();
+
+    reader.forciblyRefreshAllClusterStateSlow();
+    ref = reader.getClusterState().getCollectionRef("c1");
+    assertNull(ref);
+
+    ref = reader.getClusterState().getCollectionRef("c2");
+    assertNotNull(ref);
+    assertTrue(
+        "c2 should have been lazily loaded but is not!",
+        ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+    assertEquals(0, ref.get().getZNodeVersion());
+  }
 
-      // create new collection
-      DocCollection state =
-          new DocCollection(
-              "c1",
-              new HashMap<>(),
-              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
-              DocRouter.DEFAULT,
-              0);
-      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
-      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
-      writer.writePendingUpdates();
+  public void testGetCurrentCollections() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+
+    reader.registerCore("c1"); // listen to c1. not yet exist
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+    reader.forceUpdateCollection("c1");
+    Set<String> currentCollections = reader.getCurrentCollections();
+    assertEquals(0, currentCollections.size()); // no active collections yet
+
+    // now create both c1 (watched) and c2 (not watched)
+    DocCollection state1 =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+    DocCollection state2 =
+        new DocCollection(
+            "c2",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+
+    // do not listen to c2
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+    ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+    writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+    writer.writePendingUpdates();
+
+    reader.forceUpdateCollection("c1");
+    reader.forceUpdateCollection("c2");
+    currentCollections =
+        reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+    // loaded)

Review Comment:
   Thanks! I will try again, I ran both `./gradlew :solr:solrj:spotlessApply` and  `./gradlew :solr:core:spotlessApply` and sometimes it produces some weird wrapping...



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1171782061

   > Ok I think I'm on your side now. But I do think there is some confusion between having a `null` value in `collectionsWatches` and having a `null` `currentDoc` in the value. Can you explain that difference?
   
   For sure! There are 3 variations:
   1. `collectionWatches` does not have key for such collection - That means the collection is not being watched
   2. `collectionWatches.get("test-collection")` returns null, this should not happen, as all the entries insert into the map is non null
   3. `collectionWatches.get("test-collection")` does returns a `DocCollectionWatch` but has field `currentDoc` as `null`, that means the collection is actively watched, however either the `refreshAndWatch` is not called yet, or last fetched state indicates that the collection is not yet existed or deleted from ZK.
   
   
   > 
   > Also can we rename `currentDoc` to `currentState`? Also `DocCollectionWatch` would be better named `StatefulCollectionWatch`, because the important characteristic is that its saving the current state of the collection.
   > 
   > Thanks again for the work here!
   
   Agree thanks! Doing the renaming as suggested!


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1190645505

   Thanks so much for the work here @patsonluk , it was a long one, but I think very beneficial to the `ZkStateReader`!


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1218692060

   > FYI this has been causing NPE in ZkStateReaderTest - #966
   
   Thanks for fixing it! 🙇🏼 


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915129741


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,95 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, StatefulCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection

Review Comment:
   Good question @HoustonPutman ! I remember trying to make my method to behave the same as `updateWatchedCollection`, that it returns `true` even if:
   1. Both oldState and newState are null, this could happen after deletion when i debug
   2. newState has older version than oldState, hence no updates are made
   
   After I simplified my logic, i notice that the end result turns out to be just "whether the collection was watched" , hence i changed the javadoc to describe this exact behavior 😓 
   
   By looking at the existing `updateWatchedCollection`, it appears to only return false if the collection is no longer watched right at the beginning of the method body with exception that new state is null? (and the new class should avoid the case which `collectionWatches` changed during the call)
   
   
   Though I do share similar concerns that some places that rely on the return values of the existing method, might now seem  a bit odd ie:
   ```
   if (collectionWatches.updateDocCollection(coll, newState)) {
             updatedCollections.add(coll);
   ```
   
   I am not sure if I completely follow the original comments in the code
   ```
   // no change to state, but we might have been triggered by the addition of a
             // state watcher, so run notifications
   ```
   So is the return value actually flags whether we should run notifications (i assume it means running `constructState`?). If so, would it make sense to just call this flag `shouldConstructState` or something? 🤔  Though the `updateDocCollection` itself imo, should not really have that knowledge whether it's addition of state watcher or not, perhaps it should be the caller that should be aware of it? 



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1162477379

   > Thanks for tackling this Patson!
   > 
   > I haven't gone over everything, but took a first-pass look. I think we should probably keep the helper methods instead of sub-classing ConcurrentHashMap. But that's a separate issue than what you are trying to fix really.
   
   Yes we can discuss about that, I don't have a strong opinion about that.
   
   My design of putting it into a separate class is based on the reasoning that updating the doc collection is tightly tied with the state of the of the watcher, especially that the `collectionsWatched` contains the DocCollection state now 😄 
   
   With the helper approach, it might be slightly hard to figure out the helper actually modifies the field by inspecting the method signature alone. It's more of a personal preference that I found methods with side effect a bit hard to track sometimes 😅 
   


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r911543471


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+      assertEquals(0, ref.get().getZNodeVersion());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testGetCurrentCollections() throws Exception {
+    Path zkDir = createTempDir("testGetCurrentCollections");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // listen to c1. not yet exist
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+      reader.forceUpdateCollection("c1");
+      Set<String> currentCollections = reader.getCurrentCollections();
+      assertEquals(0, currentCollections.size()); // no active collections yet
+
+      // now create both c1 (watched) and c2 (not watched)
+      DocCollection state1 =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+      DocCollection state2 =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+
+      // do not listen to c2
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forceUpdateCollection("c1");
+      reader.forceUpdateCollection("c2");
+      currentCollections =
+          reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+      // loaded)
+      assertEquals(2, currentCollections.size());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testWatchRaceCondition() throws Exception {
+    final int RUN_COUNT = 10000;
+    Path zkDir = createTempDir("testWatchRaceCondition");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+    ExecutorService executorService =
+        ExecutorUtil.newMDCAwareSingleThreadExecutor(
+            new SolrNamedThreadFactory("zkStateReaderTest"));
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      final ZkStateReader readerRef = reader;
+      reader.createClusterStateWatchersAndUpdate();
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      // start another thread to constantly updating the state
+      final AtomicBoolean stopMutatingThread = new AtomicBoolean(false);
+      final ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      final AtomicInteger updateCounts = new AtomicInteger(0);

Review Comment:
   yep good catch!



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915038151


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   So I think Mike just wants this new class to have a single variable, the CHM, and all methods act on it. Since the CHM is thread safe (well for the uses we are using it for currently), there should be no difference to calling the CHM methods, versus calling the new `DocCollectionWatches` methods that directly call the CHM.
   
   I agree that it's probably a more easy to reason-out method, since most people aren't expecting the built-in classes to be extended. More-so, it allows developers down the line to interact directly with the CHM, while you have already written careful methods to abstract out interacting with it. So basically all you need to do is abstract out a few more of these methods, and you have a guarantee that no one is going to do something sketchy to the CHM without going through the allowed paradigms.
   
   @madrob feel free to add more if you had other reasons.
   
   Overall the change should not have to affect a whole lot of the PR. You might just need to add the following methods: `getAllCollections()` `numCollections()` and `numActiveCollections()`, and use them instead of the CHM methods throughout the rest of the file.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915031868


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,95 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, StatefulCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection

Review Comment:
   Ok it looks like this return signature contract has changed from the previous method.
   
   The previous method (`updateWatchedCollection`) returned whether the state changed. This is returning whether the new state is null. That's quite a difference. Is there a reason you did this? It should be pretty easy to add in an atomic boolean and check whether the state was changed and return that.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915420568


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -29,35 +38,68 @@
 import org.apache.solr.cloud.ZkTestServer;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocCollectionWatcher;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.util.TimeOut;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZkStateReaderTest extends SolrTestCaseJ4 {
-
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final long TIMEOUT = 30;
 
-  public void testExternalCollectionWatchedNotWatched() throws Exception {
+  private static class TestFixture implements Closeable {

Review Comment:
   You can use `getTestName()` to get the current test name as a string (inherited from LuceneTestCase) and pass that to `createTempDir` and it should all work out. Let me know if there are issues with that approach, and we can think about how to simplify if needed.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r921419678


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,95 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, StatefulCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection

Review Comment:
   so `constructState` takes in the list of collections that should be notified. It constructs the whole state, then notifies the watchers of the "updated" collections afterwards.
   
   > I am not sure if I completely follow the original comments in the code
   
   I do not think that comment still applies... The only times where the return value of this method is used is in are:
   - `forciblyRefreshAllClusterStateSlow()`
   - `forceUpdateCollection(String collection)`
   - `compareStateVersions(String coll, int version)`
   
   None of these are adding a state watcher. And every creation of a `StateWatcher` uses `new StateWatcher(collection).refreshAndWatch()`
   
   in `refreshAndWatch()` we find:
   ```java
   DocCollection newState = fetchCollectionState(coll, this);
   collectionWatches.updateDocCollection(coll, newState);
   synchronized (getUpdateLock()) {
     constructState(Collections.singleton(coll));
   }
   ```
   
   Meaning `constructState` is always called on new StateWatchers anyways... So to end, I think we can remove the `=` in `>=` for that if statement. But we do need to adhere to the original functionality of the return statement, which is "has this state changed?"



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r921461000


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +245,138 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch =
+          statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>>
+        watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      StatefulCollectionWatch finalWatch =
+          statefulWatchesByCollectionName.computeIfPresent(
+              collection,
+              (col, watch) -> {
+                DocCollection oldState = watch.currentState;
+                if (oldState == null && newState == null) {
+                  // OK, the collection not yet exist in ZK or already deleted
+                } else if (oldState == null) {
+                  if (log.isDebugEnabled()) {
+                    log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+                  }
+                  watch.currentState = newState;
+                } else if (newState == null) {
+                  log.debug("Removing cached collection state for [{}]", collection);
+                  watch.currentState = null;
+                } else { // both new and old states are non-null
+                  int oldCVersion =
+                      oldState.getPerReplicaStates() == null
+                          ? -1
+                          : oldState.getPerReplicaStates().cversion;
+                  int newCVersion =
+                      newState.getPerReplicaStates() == null
+                          ? -1
+                          : newState.getPerReplicaStates().cversion;
+                  if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                      || oldCVersion < newCVersion) {
+                    watch.currentState = newState;
+                    if (log.isDebugEnabled()) {
+                      log.debug(
+                          "Updating data for [{}] from [{}] to [{}]",
+                          collection,
+                          oldState.getZNodeVersion(),
+                          newState.getZNodeVersion());
+                    }
+                  }
+                }
+                return watch;
+              });
+      return finalWatch != null;
+    }

Review Comment:
   ```suggestion
        * @return whether the state has changed for the watched collection
        */
       private boolean updateDocCollection(String collection, DocCollection newState) {
         AtomicBoolean stateHasChanged = new AtomicBoolean(false);
         statefulWatchesByCollectionName.computeIfPresent(
             collection,
             (col, watch) -> {
               DocCollection oldState = watch.currentState;
               if (oldState == null && newState == null) {
                 // OK, the collection not yet exist in ZK or already deleted
               } else if (oldState == null) {
                 if (log.isDebugEnabled()) {
                   log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
                 }
                 watch.currentState = newState;
               } else if (newState == null) {
                 log.debug("Removing cached collection state for [{}]", collection);
                 watch.currentState = null;
               } else { // both new and old states are non-null
                 int oldCVersion =
                     oldState.getPerReplicaStates() == null
                         ? -1
                         : oldState.getPerReplicaStates().cversion;
                 int newCVersion =
                     newState.getPerReplicaStates() == null
                         ? -1
                         : newState.getPerReplicaStates().cversion;
                 if (oldState.getZNodeVersion() < newState.getZNodeVersion()
                     || oldCVersion < newCVersion) {
                   watch.currentState = newState;
                   if (log.isDebugEnabled()) {
                     log.debug(
                         "Updating data for [{}] from [{}] to [{}]",
                         collection,
                         oldState.getZNodeVersion(),
                         newState.getZNodeVersion());
                   }
                 }
               }
               stateHasChanged.set(oldState != watch.currentState);
               return watch;
             });
   
         return stateHasChanged.get();
       }
   ```



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1190542361

   @HoustonPutman done! (hopefully this is the right place 😅  https://github.com/apache/solr/pull/909/files#diff-cdd1ae1b418ea74ca689c6e68e61f4e5b4c6f77f6408a4e6cf76ed8421b24d09)


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915358372


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -540,7 +627,7 @@ private void constructState(Set<String> changedCollections) {
       log.debug(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
           collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
+          collectionWatches.activeCollections().size(),

Review Comment:
   Thanks @madrob ! There's a if `(log.isTraceEnabled()) {` later on that prints out all the names. So are you proposing to have 2 methods, one for count and one for the active set of names?



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915363090


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -29,35 +38,68 @@
 import org.apache.solr.cloud.ZkTestServer;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocCollectionWatcher;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.util.TimeOut;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZkStateReaderTest extends SolrTestCaseJ4 {
-
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final long TIMEOUT = 30;
 
-  public void testExternalCollectionWatchedNotWatched() throws Exception {
+  private static class TestFixture implements Closeable {

Review Comment:
   Thanks! I tried using `@Before` and `@After` but it's a bit tricky as I wish to call `createTempDir` with different test name therefore we will need to pass some parameters to the @Before. I looked up and there's this https://stackoverflow.com/questions/26357751/passing-parameter-to-before-setup-in-junit-test but it seems a bit complicated for here.
   
   Is it okay if we keep the current structure?



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r916221705


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +432,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.watchedCollections()) {
         DocCollection newState = fetchCollectionState(coll, null);

Review Comment:
   I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1184757807

   I have a commit to fix the return signature, could you check-off the `Allow maintainers to commit` option?


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1185019321

   @madrob @HoustonPutman I added 2 commits to better reproduce the race condition (copied the test changes to `main` branch and confirmed that it does trigger the failure). https://github.com/apache/solr/compare/24a05df012f1354ab210fa8e9a593c249f8254ad..0feb4e61bd02afdfff282b4495bf2bf8cf5050d3
   
   Seems like we have addressed all the review comments? If so, what are the next steps? Many thanks again! 😊 


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r911537402


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   Would you mind to explain some common issues with extending ConcurrentHashMap? 😊  I am totally open to moving that to another case that encapsulates it as a field, but I do want to learn a bit more! (and for that new class we probably need to have some extra synchronization?)
   
   I guess I chose this current solution as it was a concurrent hash map before, so extending it can allow all existing invocations to stay as is.
   
   



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922379034


##########
solr/solrj/src/java/org/apache/solr/common/util/CommonTestInjection.java:
##########
@@ -39,4 +45,31 @@ public static void setAdditionalProps(Map<String, String> additionalSystemProps)
   public static Map<String, String> injectAdditionalProps() {
     return additionalSystemProps;
   }
+
+  /**
+   * Set test delay (sleep) in unit of millisec
+   *
+   * @param delay delay in millisec, null to remove such delay
+   */
+  public static void setDelay(Integer delay) {
+    CommonTestInjection.delay = delay;
+  }
+
+  /**
+   * Inject an artificial delay(sleep) into the code
+   *
+   * @return true
+   */
+  public static boolean injectDelay() {
+    if (delay != null) {
+      try {
+        log.info("Start: artificial delay for {}ms", delay);
+        Thread.sleep(delay);
+        log.info("Finish: artificial delay for {}ms", delay);

Review Comment:
   Good point !



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r903157548


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentDoc : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, DocCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentDoc != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link DocCollectionWatch} if the
+     * collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      if (watch != null) {
+        DocCollection oldState = watch.currentDoc;
+        if (oldState == null && newState == null) {
+          // OK, the collection not yet exist in ZK
+        } else if (oldState == null) {
+          if (log.isDebugEnabled()) {
+            log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+          }
+          watch.currentDoc = newState;
+        } else if (newState == null) {
+          log.debug("Removing cached collection state for [{}]", collection);
+          watch.currentDoc = null;
+        } else { // both new and old states are non-null
+          int oldCVersion =
+              oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
+          int newCVersion =
+              newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
+          if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+              || oldCVersion < newCVersion) {
+            watch.currentDoc = newState;
+            if (log.isDebugEnabled()) {
+              log.debug(
+                  "Updating data for [{}] from [{}] to [{}]",
+                  collection,
+                  oldState.getZNodeVersion(),
+                  newState.getZNodeVersion());
+            }
+          }
+        }
+        return true;
+      } else {
+        return false;
+      }
+    }

Review Comment:
   That's a good point and thank you for the suggestion! 😊 
   
   Yes synchronization is necessary to guarantee higher version is being taken where there are 2 concurrent calls with different versions of `newState`  
   
   Just a really minor nit, perhaps we can even remove the `if (watch != null)` check now was it's `computeIfPresent`?



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r911534082


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -540,7 +627,7 @@ private void constructState(Set<String> changedCollections) {
       log.debug(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
           collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
+          collectionWatches.activeCollections().size(),

Review Comment:
   Thank you for the suggestion @madrob ! Yes, I struggled a little bit on this one as I first created a separate set to keep track of active collections, but once there are multiple collections, there are sync issues. As for a specialized method, can you give a bit more details on what it will look like? 



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r911547235


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+      assertEquals(0, ref.get().getZNodeVersion());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testGetCurrentCollections() throws Exception {
+    Path zkDir = createTempDir("testGetCurrentCollections");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // listen to c1. not yet exist
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+      reader.forceUpdateCollection("c1");
+      Set<String> currentCollections = reader.getCurrentCollections();
+      assertEquals(0, currentCollections.size()); // no active collections yet
+
+      // now create both c1 (watched) and c2 (not watched)
+      DocCollection state1 =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+      DocCollection state2 =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+
+      // do not listen to c2
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forceUpdateCollection("c1");
+      reader.forceUpdateCollection("c2");
+      currentCollections =
+          reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+      // loaded)
+      assertEquals(2, currentCollections.size());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testWatchRaceCondition() throws Exception {
+    final int RUN_COUNT = 10000;
+    Path zkDir = createTempDir("testWatchRaceCondition");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+    ExecutorService executorService =
+        ExecutorUtil.newMDCAwareSingleThreadExecutor(
+            new SolrNamedThreadFactory("zkStateReaderTest"));
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      final ZkStateReader readerRef = reader;
+      reader.createClusterStateWatchersAndUpdate();
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      // start another thread to constantly updating the state
+      final AtomicBoolean stopMutatingThread = new AtomicBoolean(false);
+      final ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      final AtomicInteger updateCounts = new AtomicInteger(0);
+      final AtomicReference<Exception> updateException = new AtomicReference<>();
+      executorService.submit(
+          () -> {
+            try {
+              ClusterState clusterState = readerRef.getClusterState();
+              while (!stopMutatingThread.get()) {
+                DocCollection collection = clusterState.getCollectionOrNull("c1");
+                int currentVersion = collection != null ? collection.getZNodeVersion() : 0;
+                // create new collection
+                DocCollection state =
+                    new DocCollection(
+                        "c1",
+                        new HashMap<>(),
+                        Map.of(
+                            ZkStateReader.CONFIGNAME_PROP,
+                            ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+                        DocRouter.DEFAULT,
+                        currentVersion);
+                ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+                writer.enqueueUpdate(clusterState, Collections.singletonList(wc), null);
+                clusterState = writer.writePendingUpdates();
+              }
+            } catch (Exception e) {
+              updateException.set(e);
+            }
+            return null;
+          });
+      executorService.shutdown();
+
+      for (int i = 0; i < RUN_COUNT; i++) {
+        final CountDownLatch latch = new CountDownLatch(2);
+
+        // remove itself on 2nd trigger
+        DocCollectionWatcher dummyWatcher =
+            collection -> {
+              latch.countDown();
+              return latch.getCount() == 0;
+            };
+        reader.registerDocCollectionWatcher("c1", dummyWatcher);
+        latch.await(10, TimeUnit.SECONDS);
+        reader.removeDocCollectionWatcher("c1", dummyWatcher);
+
+        boolean refLazilyLoaded = false;
+        for (int j = 0; j < 10; j++) {
+          if (reader.getClusterState().getCollectionRef("c1").isLazilyLoaded()) {
+            refLazilyLoaded = true; // it should eventually be lazily loaded
+            break;
+          }
+          int attempt = j + 1;
+          log.info("ref is not lazily loaded yet. Attempt : {}", attempt);
+
+          TimeUnit.MILLISECONDS.sleep(100);
+        }

Review Comment:
   ah didn't know that! Much cleaner with it 👍🏼 



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922290330


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -525,8 +652,14 @@ private void constructState(Set<String> changedCollections) {
     Map<String, ClusterState.CollectionRef> result = new LinkedHashMap<>();
 
     // Add collections
-    for (Map.Entry<String, DocCollection> entry : watchedCollectionStates.entrySet()) {
-      result.put(entry.getKey(), new ClusterState.CollectionRef(entry.getValue()));
+    for (Entry<String, StatefulCollectionWatch> entry :
+        collectionWatches.watchedCollectionEntries()) {
+      if (entry.getValue().currentState != null) {
+        // if the doc is null for the collection watch, then it should not be inserted into the
+        // state
+        result.putIfAbsent(

Review Comment:
   `result` starts out as an empty map, so when you loop through the keys of another map and add them to `result`, there can't be duplicates. This could honestly just be ` put()`, I think.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915363420


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -184,16 +195,210 @@ public void testWatchedCollectionCreation() throws Exception {
       writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
       writer.writePendingUpdates();
 
-      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+      assertTrue(
+          fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
 
       // reader.forceUpdateCollection("c1");
       reader.waitForState("c1", TIMEOUT, TimeUnit.SECONDS, (n, c) -> c != null);
       ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
       assertNotNull(ref);
       assertFalse(ref.isLazilyLoaded());
+    }
+  }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    try (TestFixture fixture = setupTestFixture("testForciblyRefreshAllClusterState")) {
+      ZkStateWriter writer = fixture.writer;
+      ZkStateReader reader = fixture.reader;
+
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(
+          fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched

Review Comment:
   good catch thanks!



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915129741


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,95 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, StatefulCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection

Review Comment:
   Good question @HoustonPutman ! I remember trying to make my method to behave the same as `updateWatchedCollection`, that it returns `true` even if:
   1. Both oldState and newState are null, this could happen after deletion when i debug
   2. newState has older version than oldState, hence no updates are made
   
   After I simplified my logic, i notice that the end result turns out to be just "whether the collection was watched" , hence i changed the javadoc to describe this exact behavior 😓 
   
   By looking at the existing `updateWatchedCollection`, it appears to only return false if the collection is no longer watched right at the beginning of the method body with exception that new state is null? (and the new class should avoid the case which `collectionWatches` changed during the call)
   
   
   Though I do share similar concerns that some places that rely on the return values of the existing method, might now seem  a bit odd ie:
   ```
   if (collectionWatches.updateDocCollection(coll, newState)) {
             updatedCollections.add(coll);
   ```
   
   I am not sure if I completely follow the original comments in the code
   ```
   // no change to state, but we might have been triggered by the addition of a
             // state watcher, so run notifications
   ```
   So is the return value actually flags whether we should run notifications (i assume it means running `constructState`?). If so, would it make sense to just call this flag `shouldReconstructState` or something? 🤔  



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman merged pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman merged PR #909:
URL: https://github.com/apache/solr/pull/909


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922377084


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +432,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.watchedCollections()) {
         DocCollection newState = fetchCollectionState(coll, null);

Review Comment:
   Added comment to both this iteration logic as well as `DocCollectionWatches#activeCollections`



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1180690433

   Thank you again for the helpful review @HoustonPutman and @madrob . It seems like there's this https://github.com/apache/solr/pull/909#discussion_r915129741 that is only one outstanding? @HoustonPutman 🙇🏼 
   
   I am wondering what is the next step if we resolve the last outstanding comment? Many thank again!


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1190464507

   Tests pass, this is good to go when we get the Changelog entry


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r911542761


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());

Review Comment:
   Nice suggestion! I'm on it now!



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r911530388


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +408,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.keySet()) {

Review Comment:
   Ah yes. This is one of the change that Im not 100% sure as it changes the behavior. It was buried in one of the original questions:
   
   1. In `forciblyRefreshAllClusterStateSlow`, which we are proposing to just replace `watchedCollectionStates.keySet()` with `collectionWatches.keySet()`, the reasoning is that we should forcibly update every collection that is registered in `collectionWatches`, even if previous fetch on such collection returned null (ie `watchedCollectionStates` would have no entry on it). This is a minor change of behavior but I think it is more "correct"? Would really need more experienced Solr dev to share their thoughts here 🙏🏼 



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r903157548


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentDoc : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, DocCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentDoc != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link DocCollectionWatch} if the
+     * collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      if (watch != null) {
+        DocCollection oldState = watch.currentDoc;
+        if (oldState == null && newState == null) {
+          // OK, the collection not yet exist in ZK
+        } else if (oldState == null) {
+          if (log.isDebugEnabled()) {
+            log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+          }
+          watch.currentDoc = newState;
+        } else if (newState == null) {
+          log.debug("Removing cached collection state for [{}]", collection);
+          watch.currentDoc = null;
+        } else { // both new and old states are non-null
+          int oldCVersion =
+              oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
+          int newCVersion =
+              newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
+          if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+              || oldCVersion < newCVersion) {
+            watch.currentDoc = newState;
+            if (log.isDebugEnabled()) {
+              log.debug(
+                  "Updating data for [{}] from [{}] to [{}]",
+                  collection,
+                  oldState.getZNodeVersion(),
+                  newState.getZNodeVersion());
+            }
+          }
+        }
+        return true;
+      } else {
+        return false;
+      }
+    }

Review Comment:
   That's a good point and thank you for the suggestion! 😊 
   
   Yes synchronization is necessary to guarantee higher version being taken when there are 2 concurrent calls with different versions of `newState`  
   
   Just a really minor nit, perhaps we can even remove the `if (watch != null)` check now as it's `computeIfPresent`?



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915236042


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   Thank you for the detailed explanations! @HoustonPutman 😊 
   
   I have refactored the code but somehow after the refactoring the test case `ZkStateReaderTest#testWatchRaceCondition` started to fail consistently almost every time , while I also ran it on the pre-refactor code around 10 times and it didn't fail once.
   
   I am still trying to figure out why, but meanwhile this is my change, perhaps something caught ur 👀 :
   https://github.com/fullstorydev/solr/compare/patsonluk/SOLR-16257/race-condition-can-lead-to-inconsistent-state...fullstorydev:solr:patsonluk/SOLR-16257/refactored?expand=1



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915422193


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -549,7 +636,7 @@ private void constructState(Set<String> changedCollections) {
       log.trace(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
           collectionWatches.keySet(),
-          watchedCollectionStates.keySet(),
+          collectionWatches.activeCollections().size(),

Review Comment:
   This shouldn't print the size, it should print the collection names to match the rest of the intent of this trace statement.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922392959


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -525,8 +652,14 @@ private void constructState(Set<String> changedCollections) {
     Map<String, ClusterState.CollectionRef> result = new LinkedHashMap<>();
 
     // Add collections
-    for (Map.Entry<String, DocCollection> entry : watchedCollectionStates.entrySet()) {
-      result.put(entry.getKey(), new ClusterState.CollectionRef(entry.getValue()));
+    for (Entry<String, StatefulCollectionWatch> entry :
+        collectionWatches.watchedCollectionEntries()) {
+      if (entry.getValue().currentState != null) {
+        // if the doc is null for the collection watch, then it should not be inserted into the
+        // state
+        result.putIfAbsent(

Review Comment:
   I'm scratching my head too 🤣 , as the latest code is not doing `putIfAbsent` here either. I feel like I could be just copying the original code from some earlier [commit](https://github.com/apache/solr/blame/54c5dd7d6d8b486df9132e637a6d7c5871ac4664/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java#L591)  
   
   Anyway, using `put` instead!



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r921486062


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +245,138 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch =
+          statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>>
+        watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      StatefulCollectionWatch finalWatch =
+          statefulWatchesByCollectionName.computeIfPresent(
+              collection,
+              (col, watch) -> {
+                DocCollection oldState = watch.currentState;
+                if (oldState == null && newState == null) {
+                  // OK, the collection not yet exist in ZK or already deleted
+                } else if (oldState == null) {
+                  if (log.isDebugEnabled()) {
+                    log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+                  }
+                  watch.currentState = newState;
+                } else if (newState == null) {
+                  log.debug("Removing cached collection state for [{}]", collection);
+                  watch.currentState = null;
+                } else { // both new and old states are non-null
+                  int oldCVersion =
+                      oldState.getPerReplicaStates() == null
+                          ? -1
+                          : oldState.getPerReplicaStates().cversion;
+                  int newCVersion =
+                      newState.getPerReplicaStates() == null
+                          ? -1
+                          : newState.getPerReplicaStates().cversion;
+                  if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                      || oldCVersion < newCVersion) {
+                    watch.currentState = newState;
+                    if (log.isDebugEnabled()) {
+                      log.debug(
+                          "Updating data for [{}] from [{}] to [{}]",
+                          collection,
+                          oldState.getZNodeVersion(),
+                          newState.getZNodeVersion());
+                    }
+                  }
+                }
+                return watch;
+              });
+      return finalWatch != null;
+    }

Review Comment:
   @HoustonPutman Your implementation makes perfect sense as it does reflect whether the state has been updated, which if the existing [`updateWatchedCollection` before our change](https://github.com/apache/solr/blob/7339c37ad5893ad003e3d73fa72007e1ddd60747/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java#L1951)  does behave as described in method comment, then it will be all good!
   
   Unfortunately, the original `updateWatchedCollection` basically has 2 cases which violate what it describes as `returns true if the state has changed`:
   1. If both new and old states are null (which could happen if either the collection is not yet created or is removed), it would still return `true`
   2. If the old state has version number >= the new state, even though there's no update, it would still return `true`
   
   The 2nd case in particular could be problematic, as described in the comment:
   ```
   // no change to state, but we might have been triggered by the addition of a
             // state watcher, so run notifications
   ```
   Basically the method is trying to behave differently because it has knowledge of certain behavior requirement of the caller, such reverse dependency is usually not desirable.
   
   And my original attempt was to remove such reverse dependency by simply observing what does the return value actually mean after I simplified the logic to mimic the exact return value of the original `updateWatchedCollection`. My proposed change turns out to be equally confusing 😓 



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922551286


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +246,134 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch = statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>> watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether the state has changed for the watched collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      AtomicBoolean stateHasChanged = new AtomicBoolean(false);
+      statefulWatchesByCollectionName.computeIfPresent(
+          collection,
+          (col, watch) -> {
+            DocCollection oldState = watch.currentState;
+            if (oldState == null && newState == null) {
+              // OK, the collection not yet exist in ZK or already deleted
+            } else if (oldState == null) {
+              if (log.isDebugEnabled()) {
+                log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+              }
+              watch.currentState = newState;
+            } else if (newState == null) {
+              log.debug("Removing cached collection state for [{}]", collection);
+              watch.currentState = null;
+            } else { // both new and old states are non-null
+              int oldCVersion =
+                  oldState.getPerReplicaStates() == null
+                      ? -1
+                      : oldState.getPerReplicaStates().cversion;
+              int newCVersion =
+                  newState.getPerReplicaStates() == null
+                      ? -1
+                      : newState.getPerReplicaStates().cversion;
+              if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                  || oldCVersion < newCVersion) {

Review Comment:
   @madrob  Yes I think a test case to verify different version handling makes sense!
   
   Added a new test case for that. And since we are testing `cversion`, i added PRS into the unit test (otherwise it will always be -1).
   
   Does this address your concern?
   https://github.com/apache/solr/pull/909/files#diff-c92a116c89a724226fb6efbea4c4f90ab70b40a779e1c50155c8eca646520358R227-R342



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r903157548


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentDoc : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, DocCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentDoc != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link DocCollectionWatch} if the
+     * collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      if (watch != null) {
+        DocCollection oldState = watch.currentDoc;
+        if (oldState == null && newState == null) {
+          // OK, the collection not yet exist in ZK
+        } else if (oldState == null) {
+          if (log.isDebugEnabled()) {
+            log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+          }
+          watch.currentDoc = newState;
+        } else if (newState == null) {
+          log.debug("Removing cached collection state for [{}]", collection);
+          watch.currentDoc = null;
+        } else { // both new and old states are non-null
+          int oldCVersion =
+              oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
+          int newCVersion =
+              newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
+          if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+              || oldCVersion < newCVersion) {
+            watch.currentDoc = newState;
+            if (log.isDebugEnabled()) {
+              log.debug(
+                  "Updating data for [{}] from [{}] to [{}]",
+                  collection,
+                  oldState.getZNodeVersion(),
+                  newState.getZNodeVersion());
+            }
+          }
+        }
+        return true;
+      } else {
+        return false;
+      }
+    }

Review Comment:
   That's a good point and thank you for the suggestion! 😊 
   
   Yes synchronization is necessary to guarantee higher version being taken when there are 2 concurrent calls with different versions of `newState`  
   
   Just a really minor nit, perhaps we can even remove the `if (watch != null)` check now was it's `computeIfPresent`?



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r910384430


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +408,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.keySet()) {

Review Comment:
   I don't think that we actually want any of the null collections, right?
   
   They would only be null if they stopped existing, correct? I might be wrong here



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -1640,7 +1727,6 @@ public void unregisterCore(String collection) {
           if (v == null) return null;
           if (v.coreRefCount > 0) v.coreRefCount--;
           if (v.canBeRemoved()) {
-            watchedCollectionStates.remove(collection);
             lazyCollectionStates.put(collection, new LazyCollectionRef(collection));
             reconstructState.set(true);
             return null;

Review Comment:
   So what happens when you return `null` in the compute? Does it delete the entry in the map or does it set the value to `null`?
   
   If it's the later, then we do need the null checks in all of the methods that interact with the `collectionWatches` values.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915147046


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   You covered it pretty nicely, Houston.
   
   The main concern is really about future maintainability. Right now we're handling it fine and the boundaries are clear and we're not doing anything that we're not supposed to be doing. But the risk is that at some point this class ends up getting treated like any other Map (if it gets passed or returned somewhere) and people start calling map methods on it. Then any additional internal state that we're keeping (which right now it doesn't look like there is any) can get messed up. For example, if we had a variable tracking the number of watches fired and somebody else added a new entry to the map without using our abstractions then our count would be off. So we want to expose the things meant to be exposed and encapsulate the rest.
   
   Hopefully that is clear enough, it's a lot of text and I'm not sure if I made it worse.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915129741


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,95 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, StatefulCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection

Review Comment:
   Good question @HoustonPutman ! I remember trying to make my method to behave the same as `updateWatchedCollection`, that it returns `true` even if:
   1. Both oldState and newState are null, this could happen after deletion when i debug
   2. newState has older version than oldState, hence no updates are made
   
   After I simplified my logic, i notice that the end result turns out to be just "whether the collection was watched" , hence i changed the javadoc to describe this exact behavior 😓 
   
   By looking at the existing `updateWatchedCollection`, it appears to only return false if the collection is no longer watched (with exception that new state is null?)
   
   Though I do share the concerns that some places that rely on the return values of the new method, might now seem  a bit odd ie:
   ```
   if (collectionWatches.updateDocCollection(coll, newState)) {
             updatedCollections.add(coll);
   ```
   
   I am not sure if I completely follow the original comments in the code
   ```
   // no change to state, but we might have been triggered by the addition of a
             // state watcher, so run notifications
   ```
   So is the return value actually flags whether we should run notifications (i assume it means running `constructState`?). If so, would it make sense to just call this flag `shouldReconstructState` or something? 🤔  



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915236042


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   Thank you for the detailed explanations! @HoustonPutman 😊 
   
   I have refactored the code but somehow after the refactoring the test case ZkStateReaderTest#testWatchRaceCondition started to failed consistently (while I run in on the pre-refactor one around 10 times and it didn't fail once)
   
   I am still trying to figure out why, but meanwhile this is my change, perhaps someone caught ur 👀 :
   https://github.com/fullstorydev/solr/compare/patsonluk/SOLR-16257/race-condition-can-lead-to-inconsistent-state...fullstorydev:solr:patsonluk/SOLR-16257/refactored?expand=1



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1190816118

   Thank you so much for all the great reviews and code suggestions! @HoustonPutman @madrob  🎉 


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r916132053


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -29,35 +38,68 @@
 import org.apache.solr.cloud.ZkTestServer;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocCollectionWatcher;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.util.TimeOut;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZkStateReaderTest extends SolrTestCaseJ4 {
-
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final long TIMEOUT = 30;
 
-  public void testExternalCollectionWatchedNotWatched() throws Exception {
+  private static class TestFixture implements Closeable {

Review Comment:
   Ah didn't know that! Thank you for the useful pointer!



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r916134059


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -549,7 +636,7 @@ private void constructState(Set<String> changedCollections) {
       log.trace(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
           collectionWatches.keySet(),
-          watchedCollectionStates.keySet(),
+          collectionWatches.activeCollections().size(),

Review Comment:
   Thanks it was corrected



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r910461025


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());

Review Comment:
   Can we pull this into a common setup method so the logic isn't repeated in every single test method?



##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+      assertEquals(0, ref.get().getZNodeVersion());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testGetCurrentCollections() throws Exception {
+    Path zkDir = createTempDir("testGetCurrentCollections");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // listen to c1. not yet exist
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+      reader.forceUpdateCollection("c1");
+      Set<String> currentCollections = reader.getCurrentCollections();
+      assertEquals(0, currentCollections.size()); // no active collections yet
+
+      // now create both c1 (watched) and c2 (not watched)
+      DocCollection state1 =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+      DocCollection state2 =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+
+      // do not listen to c2
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forceUpdateCollection("c1");
+      reader.forceUpdateCollection("c2");
+      currentCollections =
+          reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+      // loaded)
+      assertEquals(2, currentCollections.size());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testWatchRaceCondition() throws Exception {
+    final int RUN_COUNT = 10000;
+    Path zkDir = createTempDir("testWatchRaceCondition");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+    ExecutorService executorService =
+        ExecutorUtil.newMDCAwareSingleThreadExecutor(
+            new SolrNamedThreadFactory("zkStateReaderTest"));
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      final ZkStateReader readerRef = reader;
+      reader.createClusterStateWatchersAndUpdate();
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      // start another thread to constantly updating the state
+      final AtomicBoolean stopMutatingThread = new AtomicBoolean(false);
+      final ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      final AtomicInteger updateCounts = new AtomicInteger(0);
+      final AtomicReference<Exception> updateException = new AtomicReference<>();
+      executorService.submit(
+          () -> {
+            try {
+              ClusterState clusterState = readerRef.getClusterState();
+              while (!stopMutatingThread.get()) {
+                DocCollection collection = clusterState.getCollectionOrNull("c1");
+                int currentVersion = collection != null ? collection.getZNodeVersion() : 0;
+                // create new collection
+                DocCollection state =
+                    new DocCollection(
+                        "c1",
+                        new HashMap<>(),
+                        Map.of(
+                            ZkStateReader.CONFIGNAME_PROP,
+                            ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+                        DocRouter.DEFAULT,
+                        currentVersion);
+                ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+                writer.enqueueUpdate(clusterState, Collections.singletonList(wc), null);
+                clusterState = writer.writePendingUpdates();
+              }
+            } catch (Exception e) {
+              updateException.set(e);
+            }
+            return null;
+          });
+      executorService.shutdown();
+
+      for (int i = 0; i < RUN_COUNT; i++) {
+        final CountDownLatch latch = new CountDownLatch(2);
+
+        // remove itself on 2nd trigger
+        DocCollectionWatcher dummyWatcher =
+            collection -> {
+              latch.countDown();
+              return latch.getCount() == 0;
+            };
+        reader.registerDocCollectionWatcher("c1", dummyWatcher);
+        latch.await(10, TimeUnit.SECONDS);
+        reader.removeDocCollectionWatcher("c1", dummyWatcher);
+
+        boolean refLazilyLoaded = false;
+        for (int j = 0; j < 10; j++) {
+          if (reader.getClusterState().getCollectionRef("c1").isLazilyLoaded()) {
+            refLazilyLoaded = true; // it should eventually be lazily loaded
+            break;
+          }
+          int attempt = j + 1;
+          log.info("ref is not lazily loaded yet. Attempt : {}", attempt);
+
+          TimeUnit.MILLISECONDS.sleep(100);
+        }

Review Comment:
   TimeOut.waitFor



##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+      assertEquals(0, ref.get().getZNodeVersion());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testGetCurrentCollections() throws Exception {
+    Path zkDir = createTempDir("testGetCurrentCollections");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // listen to c1. not yet exist
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+      reader.forceUpdateCollection("c1");
+      Set<String> currentCollections = reader.getCurrentCollections();
+      assertEquals(0, currentCollections.size()); // no active collections yet
+
+      // now create both c1 (watched) and c2 (not watched)
+      DocCollection state1 =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+      DocCollection state2 =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+
+      // do not listen to c2
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forceUpdateCollection("c1");
+      reader.forceUpdateCollection("c2");
+      currentCollections =
+          reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+      // loaded)
+      assertEquals(2, currentCollections.size());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testWatchRaceCondition() throws Exception {
+    final int RUN_COUNT = 10000;
+    Path zkDir = createTempDir("testWatchRaceCondition");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+    ExecutorService executorService =
+        ExecutorUtil.newMDCAwareSingleThreadExecutor(
+            new SolrNamedThreadFactory("zkStateReaderTest"));
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      final ZkStateReader readerRef = reader;
+      reader.createClusterStateWatchersAndUpdate();
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      // start another thread to constantly updating the state
+      final AtomicBoolean stopMutatingThread = new AtomicBoolean(false);
+      final ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      final AtomicInteger updateCounts = new AtomicInteger(0);

Review Comment:
   unused?



##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+      assertEquals(0, ref.get().getZNodeVersion());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testGetCurrentCollections() throws Exception {
+    Path zkDir = createTempDir("testGetCurrentCollections");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // listen to c1. not yet exist
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+      reader.forceUpdateCollection("c1");
+      Set<String> currentCollections = reader.getCurrentCollections();
+      assertEquals(0, currentCollections.size()); // no active collections yet
+
+      // now create both c1 (watched) and c2 (not watched)
+      DocCollection state1 =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+      DocCollection state2 =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+
+      // do not listen to c2
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forceUpdateCollection("c1");
+      reader.forceUpdateCollection("c2");
+      currentCollections =
+          reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+      // loaded)
+      assertEquals(2, currentCollections.size());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testWatchRaceCondition() throws Exception {
+    final int RUN_COUNT = 10000;
+    Path zkDir = createTempDir("testWatchRaceCondition");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+    ExecutorService executorService =
+        ExecutorUtil.newMDCAwareSingleThreadExecutor(
+            new SolrNamedThreadFactory("zkStateReaderTest"));
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      final ZkStateReader readerRef = reader;
+      reader.createClusterStateWatchersAndUpdate();
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      // start another thread to constantly updating the state
+      final AtomicBoolean stopMutatingThread = new AtomicBoolean(false);
+      final ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      final AtomicInteger updateCounts = new AtomicInteger(0);
+      final AtomicReference<Exception> updateException = new AtomicReference<>();
+      executorService.submit(
+          () -> {
+            try {
+              ClusterState clusterState = readerRef.getClusterState();
+              while (!stopMutatingThread.get()) {
+                DocCollection collection = clusterState.getCollectionOrNull("c1");
+                int currentVersion = collection != null ? collection.getZNodeVersion() : 0;
+                // create new collection
+                DocCollection state =
+                    new DocCollection(
+                        "c1",
+                        new HashMap<>(),
+                        Map.of(
+                            ZkStateReader.CONFIGNAME_PROP,
+                            ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+                        DocRouter.DEFAULT,
+                        currentVersion);
+                ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+                writer.enqueueUpdate(clusterState, Collections.singletonList(wc), null);
+                clusterState = writer.writePendingUpdates();
+              }
+            } catch (Exception e) {
+              updateException.set(e);
+            }
+            return null;
+          });
+      executorService.shutdown();
+
+      for (int i = 0; i < RUN_COUNT; i++) {
+        final CountDownLatch latch = new CountDownLatch(2);
+
+        // remove itself on 2nd trigger
+        DocCollectionWatcher dummyWatcher =
+            collection -> {
+              latch.countDown();
+              return latch.getCount() == 0;
+            };
+        reader.registerDocCollectionWatcher("c1", dummyWatcher);
+        latch.await(10, TimeUnit.SECONDS);

Review Comment:
   need to check return value here



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -540,7 +627,7 @@ private void constructState(Set<String> changedCollections) {
       log.debug(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
           collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
+          collectionWatches.activeCollections().size(),

Review Comment:
   I would create a specialized method for this that doesn't involve collecting into an intermediate set.



##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+      assertEquals(0, ref.get().getZNodeVersion());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testGetCurrentCollections() throws Exception {
+    Path zkDir = createTempDir("testGetCurrentCollections");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // listen to c1. not yet exist
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+      reader.forceUpdateCollection("c1");
+      Set<String> currentCollections = reader.getCurrentCollections();
+      assertEquals(0, currentCollections.size()); // no active collections yet
+
+      // now create both c1 (watched) and c2 (not watched)
+      DocCollection state1 =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+      DocCollection state2 =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+
+      // do not listen to c2
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forceUpdateCollection("c1");
+      reader.forceUpdateCollection("c2");
+      currentCollections =
+          reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+      // loaded)
+      assertEquals(2, currentCollections.size());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testWatchRaceCondition() throws Exception {
+    final int RUN_COUNT = 10000;
+    Path zkDir = createTempDir("testWatchRaceCondition");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+    ExecutorService executorService =
+        ExecutorUtil.newMDCAwareSingleThreadExecutor(
+            new SolrNamedThreadFactory("zkStateReaderTest"));
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      final ZkStateReader readerRef = reader;
+      reader.createClusterStateWatchersAndUpdate();
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      // start another thread to constantly updating the state
+      final AtomicBoolean stopMutatingThread = new AtomicBoolean(false);
+      final ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      final AtomicInteger updateCounts = new AtomicInteger(0);
+      final AtomicReference<Exception> updateException = new AtomicReference<>();
+      executorService.submit(
+          () -> {
+            try {
+              ClusterState clusterState = readerRef.getClusterState();
+              while (!stopMutatingThread.get()) {
+                DocCollection collection = clusterState.getCollectionOrNull("c1");
+                int currentVersion = collection != null ? collection.getZNodeVersion() : 0;
+                // create new collection
+                DocCollection state =
+                    new DocCollection(
+                        "c1",
+                        new HashMap<>(),
+                        Map.of(
+                            ZkStateReader.CONFIGNAME_PROP,
+                            ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+                        DocRouter.DEFAULT,
+                        currentVersion);
+                ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+                writer.enqueueUpdate(clusterState, Collections.singletonList(wc), null);
+                clusterState = writer.writePendingUpdates();
+              }
+            } catch (Exception e) {
+              updateException.set(e);
+            }
+            return null;
+          });
+      executorService.shutdown();
+
+      for (int i = 0; i < RUN_COUNT; i++) {
+        final CountDownLatch latch = new CountDownLatch(2);
+
+        // remove itself on 2nd trigger
+        DocCollectionWatcher dummyWatcher =
+            collection -> {
+              latch.countDown();
+              return latch.getCount() == 0;
+            };
+        reader.registerDocCollectionWatcher("c1", dummyWatcher);
+        latch.await(10, TimeUnit.SECONDS);
+        reader.removeDocCollectionWatcher("c1", dummyWatcher);
+
+        boolean refLazilyLoaded = false;
+        for (int j = 0; j < 10; j++) {
+          if (reader.getClusterState().getCollectionRef("c1").isLazilyLoaded()) {
+            refLazilyLoaded = true; // it should eventually be lazily loaded
+            break;
+          }
+          int attempt = j + 1;
+          log.info("ref is not lazily loaded yet. Attempt : {}", attempt);
+
+          TimeUnit.MILLISECONDS.sleep(100);
+        }
+        assert (refLazilyLoaded);

Review Comment:
   junit assert



##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+      assertEquals(0, ref.get().getZNodeVersion());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testGetCurrentCollections() throws Exception {
+    Path zkDir = createTempDir("testGetCurrentCollections");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // listen to c1. not yet exist
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+      reader.forceUpdateCollection("c1");
+      Set<String> currentCollections = reader.getCurrentCollections();
+      assertEquals(0, currentCollections.size()); // no active collections yet
+
+      // now create both c1 (watched) and c2 (not watched)
+      DocCollection state1 =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+      DocCollection state2 =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+
+      // do not listen to c2
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forceUpdateCollection("c1");
+      reader.forceUpdateCollection("c2");
+      currentCollections =
+          reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+      // loaded)
+      assertEquals(2, currentCollections.size());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testWatchRaceCondition() throws Exception {
+    final int RUN_COUNT = 10000;
+    Path zkDir = createTempDir("testWatchRaceCondition");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+    ExecutorService executorService =
+        ExecutorUtil.newMDCAwareSingleThreadExecutor(
+            new SolrNamedThreadFactory("zkStateReaderTest"));
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      final ZkStateReader readerRef = reader;
+      reader.createClusterStateWatchersAndUpdate();
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      // start another thread to constantly updating the state
+      final AtomicBoolean stopMutatingThread = new AtomicBoolean(false);
+      final ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      final AtomicInteger updateCounts = new AtomicInteger(0);
+      final AtomicReference<Exception> updateException = new AtomicReference<>();
+      executorService.submit(
+          () -> {
+            try {
+              ClusterState clusterState = readerRef.getClusterState();
+              while (!stopMutatingThread.get()) {
+                DocCollection collection = clusterState.getCollectionOrNull("c1");
+                int currentVersion = collection != null ? collection.getZNodeVersion() : 0;
+                // create new collection
+                DocCollection state =
+                    new DocCollection(
+                        "c1",
+                        new HashMap<>(),
+                        Map.of(
+                            ZkStateReader.CONFIGNAME_PROP,
+                            ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+                        DocRouter.DEFAULT,
+                        currentVersion);
+                ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+                writer.enqueueUpdate(clusterState, Collections.singletonList(wc), null);
+                clusterState = writer.writePendingUpdates();
+              }
+            } catch (Exception e) {
+              updateException.set(e);
+            }
+            return null;
+          });
+      executorService.shutdown();
+
+      for (int i = 0; i < RUN_COUNT; i++) {
+        final CountDownLatch latch = new CountDownLatch(2);
+
+        // remove itself on 2nd trigger
+        DocCollectionWatcher dummyWatcher =
+            collection -> {
+              latch.countDown();
+              return latch.getCount() == 0;
+            };
+        reader.registerDocCollectionWatcher("c1", dummyWatcher);
+        latch.await(10, TimeUnit.SECONDS);
+        reader.removeDocCollectionWatcher("c1", dummyWatcher);
+
+        boolean refLazilyLoaded = false;
+        for (int j = 0; j < 10; j++) {
+          if (reader.getClusterState().getCollectionRef("c1").isLazilyLoaded()) {
+            refLazilyLoaded = true; // it should eventually be lazily loaded
+            break;
+          }
+          int attempt = j + 1;
+          log.info("ref is not lazily loaded yet. Attempt : {}", attempt);
+
+          TimeUnit.MILLISECONDS.sleep(100);
+        }
+        assert (refLazilyLoaded);
+      }
+
+      stopMutatingThread.set(true);
+      if (updateException.get() != null) {
+        throw (updateException.get());
+      }
+
+    } finally {
+      IOUtils.close(reader, zkClient);
+      executorService.awaitTermination(10, TimeUnit.SECONDS);

Review Comment:
   ExecutorUtil.awaitTermination



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -1640,7 +1727,6 @@ public void unregisterCore(String collection) {
           if (v == null) return null;
           if (v.coreRefCount > 0) v.coreRefCount--;
           if (v.canBeRemoved()) {
-            watchedCollectionStates.remove(collection);
             lazyCollectionStates.put(collection, new LazyCollectionRef(collection));
             reconstructState.set(true);
             return null;

Review Comment:
   it removes the entry



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   extending CHM can get kind of trappy, it's usually a better idea to encapsulate a CHM as a field inside of this class and then create delegate methods where necessary.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922379526


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -1919,9 +2051,9 @@ public void removeDocCollectionWatcher(String collection, DocCollectionWatcher w
           if (v == null) return null;
           v.stateWatchers.remove(watcher);
           if (v.canBeRemoved()) {
-            watchedCollectionStates.remove(collection);
             lazyCollectionStates.put(collection, new LazyCollectionRef(collection));
             reconstructState.set(true);
+            CommonTestInjection.injectDelay(); // To unit test race condition

Review Comment:
   O missed that one. Thanks for catching!



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r921503461


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +245,138 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch =
+          statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>>
+        watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      StatefulCollectionWatch finalWatch =
+          statefulWatchesByCollectionName.computeIfPresent(
+              collection,
+              (col, watch) -> {
+                DocCollection oldState = watch.currentState;
+                if (oldState == null && newState == null) {
+                  // OK, the collection not yet exist in ZK or already deleted
+                } else if (oldState == null) {
+                  if (log.isDebugEnabled()) {
+                    log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+                  }
+                  watch.currentState = newState;
+                } else if (newState == null) {
+                  log.debug("Removing cached collection state for [{}]", collection);
+                  watch.currentState = null;
+                } else { // both new and old states are non-null
+                  int oldCVersion =
+                      oldState.getPerReplicaStates() == null
+                          ? -1
+                          : oldState.getPerReplicaStates().cversion;
+                  int newCVersion =
+                      newState.getPerReplicaStates() == null
+                          ? -1
+                          : newState.getPerReplicaStates().cversion;
+                  if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                      || oldCVersion < newCVersion) {
+                    watch.currentState = newState;
+                    if (log.isDebugEnabled()) {
+                      log.debug(
+                          "Updating data for [{}] from [{}] to [{}]",
+                          collection,
+                          oldState.getZNodeVersion(),
+                          newState.getZNodeVersion());
+                    }
+                  }
+                }
+                return watch;
+              });
+      return finalWatch != null;
+    }

Review Comment:
   > 2. If the old state has version number >= the new state, even though there's no update, it would still return true
   
   As I mentioned in https://github.com/apache/solr/pull/909#discussion_r921419678, I don't think that comment is correct, so we should be good to go with the new behavior.
   
   > 1.  If both new and old states are null (which could happen if either the collection is not yet created or is removed), it would still return true
   
   It's confusing but I don't think that is the case. Looking at the relevant part of the old implementation:
   
   ```java
    if (oldState == null) {
           if (watchedCollectionStates.putIfAbsent(coll, newState) == null) {
             if (log.isDebugEnabled()) {
               log.debug("Add data for [{}] ver [{}]", coll, newState.getZNodeVersion());
             }
             updated = true;
             break;
           }
         }
   ```
   
   [putIfAbsent()](https://docs.oracle.com/javase/8/docs/api/java/util/Map.html#putIfAbsent-K-V-) returns the previous value associated with that key, so `null` means that there was no previous value and the `putIfAbsent()` successfully executed. It doesn't actually check what the `newState` is.
   
   So it would return `true` if the old state was null, no matter what. We probably don't want to keep that functionality. No need to update collections if a state went from `null` -> `null`.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1190426126

   @patsonluk I think this is good to go! I'll add an entry in the changelog, then commit and backport this! (Gonna run the full test suite first though)


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1184771713

   Ahh thanks for reminding me about that @cpoerschke!


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1169412289

   Friendly ping @HoustonPutman . Wondering if you have any extra thoughts about this PR? 😊 


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915236042


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   Thank you for the detailed explanations! @HoustonPutman 😊 
   
   I have refactored the code but somehow after the refactoring the test case ZkStateReaderTest#testWatchRaceCondition started to failed consistently almost every time , while I also ran it on the pre-refactor one around 10 times and it didn't fail once.
   
   I am still trying to figure out why, but meanwhile this is my change, perhaps someone caught ur 👀 :
   https://github.com/fullstorydev/solr/compare/patsonluk/SOLR-16257/race-condition-can-lead-to-inconsistent-state...fullstorydev:solr:patsonluk/SOLR-16257/refactored?expand=1



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915236042


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   Thank you for the detailed explanations! @HoustonPutman 😊 
   
   ~I have refactored the code but somehow after the refactoring the test case `ZkStateReaderTest#testWatchRaceCondition` started to fail consistently almost every time , while I also ran it on the pre-refactor code around 10 times and it didn't fail once.~
   
   ~I am still trying to figure out why, but meanwhile this is my change, perhaps something caught ur 👀 :
   https://github.com/fullstorydev/solr/compare/patsonluk/SOLR-16257/race-condition-can-lead-to-inconsistent-state...fullstorydev:solr:patsonluk/SOLR-16257/refactored?expand=1~
   
   Nm! Found the cause, I used a static final field, while it should have been non static 😓 



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r916198593


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +245,125 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch<DocCollectionWatcher>>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch<DocCollectionWatcher> watch =
+          statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());

Review Comment:
   we dun have to but it's just a bit safer to outright prevent it 😊 



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r916221594


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +432,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.watchedCollections()) {
         DocCollection newState = fetchCollectionState(coll, null);

Review Comment:
   This should be okay as the watchedCollections is the key set (unmodifable) of a ConcurrentHashMap? Even if an collection entry is added/removed during the iteration, it is not critical to this logic.



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +432,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.watchedCollections()) {
         DocCollection newState = fetchCollectionState(coll, null);

Review Comment:
   @sonatype-lift ignore



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915426740


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +245,125 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch<DocCollectionWatcher>>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch<DocCollectionWatcher> watch =
+          statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());

Review Comment:
   Do we really need to wrap this in an unmodifiable set? it's private, we could javadoc asking callers to not modify this set.



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentDoc : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, DocCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentDoc != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link DocCollectionWatch} if the
+     * collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      if (watch != null) {
+        DocCollection oldState = watch.currentDoc;
+        if (oldState == null && newState == null) {
+          // OK, the collection not yet exist in ZK
+        } else if (oldState == null) {
+          if (log.isDebugEnabled()) {
+            log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+          }
+          watch.currentDoc = newState;
+        } else if (newState == null) {
+          log.debug("Removing cached collection state for [{}]", collection);
+          watch.currentDoc = null;
+        } else { // both new and old states are non-null
+          int oldCVersion =
+              oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
+          int newCVersion =
+              newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
+          if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+              || oldCVersion < newCVersion) {
+            watch.currentDoc = newState;
+            if (log.isDebugEnabled()) {
+              log.debug(
+                  "Updating data for [{}] from [{}] to [{}]",
+                  collection,
+                  oldState.getZNodeVersion(),
+                  newState.getZNodeVersion());
+            }
+          }
+        }
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  private static class DocCollectionWatch<T> extends CollectionWatch<T> {

Review Comment:
   If this is only ever used with `T = DocCollectionWatcher` then we might as well declare that explicitly and save ourselves typing on the declarations? There might be other usages that I'm not seeing though. I don't think this is something that's worth trying to make generic (yet?)
   ```suggestion
     private static class DocCollectionWatch extends CollectionWatch<DocCollectionWatcher> {
   ```



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915032474


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +408,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.keySet()) {

Review Comment:
   Yeah it's not the end of the world to look at the null states as well probably, since this is the `forciblyRefreshAllClusterStateSlow()` method.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915236042


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {

Review Comment:
   Thank you for the detailed explanations! @HoustonPutman 😊 
   
   I have refactored the code but somehow after the refactoring the test case ZkStateReaderTest#testWatchRaceCondition started to failed consistently (while I also ran it on the pre-refactor one around 10 times and it didn't fail once)
   
   I am still trying to figure out why, but meanwhile this is my change, perhaps someone caught ur 👀 :
   https://github.com/fullstorydev/solr/compare/patsonluk/SOLR-16257/race-condition-can-lead-to-inconsistent-state...fullstorydev:solr:patsonluk/SOLR-16257/refactored?expand=1



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915138036


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -184,16 +195,210 @@ public void testWatchedCollectionCreation() throws Exception {
       writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
       writer.writePendingUpdates();
 
-      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+      assertTrue(
+          fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
 
       // reader.forceUpdateCollection("c1");
       reader.waitForState("c1", TIMEOUT, TimeUnit.SECONDS, (n, c) -> c != null);
       ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
       assertNotNull(ref);
       assertFalse(ref.isLazilyLoaded());
+    }
+  }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    try (TestFixture fixture = setupTestFixture("testForciblyRefreshAllClusterState")) {
+      ZkStateWriter writer = fixture.writer;
+      ZkStateReader reader = fixture.reader;
+
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(
+          fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched

Review Comment:
   JUnit assert



##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -29,35 +38,68 @@
 import org.apache.solr.cloud.ZkTestServer;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocCollectionWatcher;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.util.TimeOut;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZkStateReaderTest extends SolrTestCaseJ4 {
-
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final long TIMEOUT = 30;
 
-  public void testExternalCollectionWatchedNotWatched() throws Exception {
+  private static class TestFixture implements Closeable {

Review Comment:
   This is an interesting approach. I meant to put things into a JUnit `@Before` and `@After` method, which is going to be more idiomatic, I think.



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -540,7 +627,7 @@ private void constructState(Set<String> changedCollections) {
       log.debug(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
           collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
+          collectionWatches.activeCollections().size(),

Review Comment:
   ```
   private long activeSize() {
          return this.entrySet().stream()
              .filter(entry -> entry.getValue().currentState != null)
              .count();
   }
   ```



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r915385252


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +432,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.watchedCollections()) {
         DocCollection newState = fetchCollectionState(coll, null);

Review Comment:
   *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `ZkStateReader.forciblyRefreshAllClusterStateSlow()` indirectly reads with synchronization from `noggit.JSONParser.devNull.end`. Potentially races with unsynchronized write in method `ZkStateReader.compareStateVersions(...)`.
    Reporting because this access may occur on a background thread.
   
   Reply with *"**@sonatype-lift help**"* for info about LiftBot commands.
   Reply with *"**@sonatype-lift ignore**"* to tell LiftBot to leave out the above finding from this PR.
   Reply with *"**@sonatype-lift ignoreall**"* to tell LiftBot to leave out all the findings from this PR and from the status bar in Github.
   
   When talking to LiftBot, you need to **refresh** the page to see its response. [Click here](https://help.sonatype.com/lift/talking-to-lift) to get to know more about LiftBot commands.
   
   ---
   
   Was this a good recommendation?
   [ [🙁 Not relevant](https://www.sonatype.com/lift-comment-rating?comment=292955795&lift_comment_rating=1) ] - [ [😕 Won't fix](https://www.sonatype.com/lift-comment-rating?comment=292955795&lift_comment_rating=2) ] - [ [😑 Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=292955795&lift_comment_rating=3) ] - [ [🙂 Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=292955795&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=292955795&lift_comment_rating=5) ]



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r922256956


##########
solr/solrj/src/java/org/apache/solr/common/util/CommonTestInjection.java:
##########
@@ -39,4 +45,31 @@ public static void setAdditionalProps(Map<String, String> additionalSystemProps)
   public static Map<String, String> injectAdditionalProps() {
     return additionalSystemProps;
   }
+
+  /**
+   * Set test delay (sleep) in unit of millisec
+   *
+   * @param delay delay in millisec, null to remove such delay
+   */
+  public static void setDelay(Integer delay) {
+    CommonTestInjection.delay = delay;
+  }
+
+  /**
+   * Inject an artificial delay(sleep) into the code
+   *
+   * @return true
+   */
+  public static boolean injectDelay() {
+    if (delay != null) {
+      try {
+        log.info("Start: artificial delay for {}ms", delay);
+        Thread.sleep(delay);
+        log.info("Finish: artificial delay for {}ms", delay);

Review Comment:
   nit: move this to a finally block so that we still log even if we were interrupted.



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -1919,9 +2051,9 @@ public void removeDocCollectionWatcher(String collection, DocCollectionWatcher w
           if (v == null) return null;
           v.stateWatchers.remove(watcher);
           if (v.canBeRemoved()) {
-            watchedCollectionStates.remove(collection);
             lazyCollectionStates.put(collection, new LazyCollectionRef(collection));
             reconstructState.set(true);
+            CommonTestInjection.injectDelay(); // To unit test race condition

Review Comment:
   our common pattern is to hide this behind a java `assert`. See also https://github.com/apache/solr/blob/main/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java#L183



##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -29,171 +38,387 @@
 import org.apache.solr.cloud.ZkTestServer;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocCollectionWatcher;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.CommonTestInjection;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.util.TimeOut;
+import org.junit.After;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZkStateReaderTest extends SolrTestCaseJ4 {
-
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final long TIMEOUT = 30;
 
-  public void testExternalCollectionWatchedNotWatched() throws Exception {
-    Path zkDir = createTempDir("testExternalCollectionWatchedNotWatched");
-    ZkTestServer server = new ZkTestServer(zkDir);
-    SolrZkClient zkClient = null;
-    ZkStateReader reader = null;
-
-    try {
-      server.run();
-
-      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
-      ZkController.createClusterZkNodes(zkClient);
-
-      reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-
-      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
-
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-
-      // create new collection
-      ZkWriteCommand c1 =
-          new ZkWriteCommand(
-              "c1",
-              new DocCollection(
-                  "c1",
-                  new HashMap<>(),
-                  Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
-                  DocRouter.DEFAULT,
-                  0));
-      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(c1), null);
-      writer.writePendingUpdates();
-      reader.forceUpdateCollection("c1");
-
-      assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
-      reader.registerCore("c1");
-      assertFalse(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
-      reader.unregisterCore("c1");
-      assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
+  private static class TestFixture implements Closeable {
+    private final ZkTestServer server;
+    private final SolrZkClient zkClient;
+    private final ZkStateReader reader;
+    private final ZkStateWriter writer;
+
+    private TestFixture(
+        ZkTestServer server, SolrZkClient zkClient, ZkStateReader reader, ZkStateWriter writer) {
+      this.server = server;
+      this.zkClient = zkClient;
+      this.reader = reader;
+      this.writer = writer;
+    }
 
-    } finally {
+    @Override
+    public void close() throws IOException {
       IOUtils.close(reader, zkClient);
-      server.shutdown();
+      try {
+        server.shutdown();
+      } catch (InterruptedException e) {
+        // ok. Shutting down anyway
+      }
     }
   }
 
-  public void testCollectionStateWatcherCaching() throws Exception {
-    Path zkDir = createTempDir("testCollectionStateWatcherCaching");
-
-    ZkTestServer server = new ZkTestServer(zkDir);
+  private TestFixture fixture = null;
 
-    SolrZkClient zkClient = null;
-    ZkStateReader reader = null;
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    fixture = setupTestFixture(getTestName());
+  }
 
-    try {
-      server.run();
-
-      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
-      ZkController.createClusterZkNodes(zkClient);
-
-      reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-
-      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
-      DocCollection state =
-          new DocCollection(
-              "c1",
-              new HashMap<>(),
-              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
-              DocRouter.DEFAULT,
-              0);
-      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
-      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
-      writer.writePendingUpdates();
-      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
-      reader.waitForState(
-          "c1", 1, TimeUnit.SECONDS, (liveNodes, collectionState) -> collectionState != null);
-
-      Map<String, Object> props = new HashMap<>();
-      props.put("x", "y");
-      props.put(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME);
-      state = new DocCollection("c1", new HashMap<>(), props, DocRouter.DEFAULT, 0);
-      wc = new ZkWriteCommand("c1", state);
-      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
-      writer.writePendingUpdates();
-
-      boolean found = false;
-      TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
-      while (!timeOut.hasTimedOut()) {
-        DocCollection c1 = reader.getClusterState().getCollection("c1");
-        if ("y".equals(c1.getStr("x"))) {
-          found = true;
-          break;
-        }
-      }
-      assertTrue("Could not find updated property in collection c1 even after 5 seconds", found);
-    } finally {
-      IOUtils.close(reader, zkClient);
-      server.shutdown();
+  @After
+  public void tearDown() throws Exception {
+    if (fixture != null) {
+      fixture.close();
     }
+    super.tearDown();
   }
 
-  public void testWatchedCollectionCreation() throws Exception {
-    Path zkDir = createTempDir("testWatchedCollectionCreation");
-
+  private static TestFixture setupTestFixture(String testPrefix) throws Exception {
+    Path zkDir = createTempDir(testPrefix);
     ZkTestServer server = new ZkTestServer(zkDir);
+    server.run();
+    SolrZkClient zkClient =
+        new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+    ZkController.createClusterZkNodes(zkClient);
 
-    SolrZkClient zkClient = null;
-    ZkStateReader reader = null;
-
-    try {
-      server.run();
+    ZkStateReader reader = new ZkStateReader(zkClient);
+    reader.createClusterStateWatchersAndUpdate();
 
-      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
-      ZkController.createClusterZkNodes(zkClient);
+    ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
 
-      reader = new ZkStateReader(zkClient);
-      reader.createClusterStateWatchersAndUpdate();
-      reader.registerCore("c1");
+    return new TestFixture(server, zkClient, reader, writer);
+  }
 
-      // Initially there should be no c1 collection.
-      assertNull(reader.getClusterState().getCollectionRef("c1"));
+  public void testExternalCollectionWatchedNotWatched() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+    // create new collection
+    ZkWriteCommand c1 =
+        new ZkWriteCommand(
+            "c1",
+            new DocCollection(
+                "c1",
+                new HashMap<>(),
+                Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+                DocRouter.DEFAULT,
+                0));
+
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(c1), null);
+    writer.writePendingUpdates();
+    reader.forceUpdateCollection("c1");
+
+    assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
+    reader.registerCore("c1");
+    assertFalse(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
+    reader.unregisterCore("c1");
+    assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
+  }
 
-      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
-      reader.forceUpdateCollection("c1");
+  public void testCollectionStateWatcherCaching() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+    DocCollection state =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+    assertTrue(fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+    reader.waitForState(
+        "c1", 1, TimeUnit.SECONDS, (liveNodes, collectionState) -> collectionState != null);
+
+    Map<String, Object> props = new HashMap<>();
+    props.put("x", "y");
+    props.put(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME);
+    state = new DocCollection("c1", new HashMap<>(), props, DocRouter.DEFAULT, 0);
+    wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+
+    boolean found = false;
+    TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    while (!timeOut.hasTimedOut()) {
+      DocCollection c1 = reader.getClusterState().getCollection("c1");
+      if ("y".equals(c1.getStr("x"))) {
+        found = true;
+        break;
+      }
+    }
+    assertTrue("Could not find updated property in collection c1 even after 5 seconds", found);
+  }
 
-      // Still no c1 collection, despite a collection path.
-      assertNull(reader.getClusterState().getCollectionRef("c1"));
+  public void testWatchedCollectionCreation() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+
+    reader.registerCore("c1");
+
+    // Initially there should be no c1 collection.
+    assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+    reader.forceUpdateCollection("c1");
+
+    // Still no c1 collection, despite a collection path.
+    assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+    // create new collection
+    DocCollection state =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+
+    assertTrue(fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+    // reader.forceUpdateCollection("c1");
+    reader.waitForState("c1", TIMEOUT, TimeUnit.SECONDS, (n, c) -> c != null);
+    ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+    assertNotNull(ref);
+    assertFalse(ref.isLazilyLoaded());
+  }
 
-      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+
+    reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+    reader.forciblyRefreshAllClusterStateSlow();
+    // Initially there should be no c1 collection.
+    assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+    // create new collection
+    DocCollection state =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+
+    assertTrue(fixture.zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+    reader.forciblyRefreshAllClusterStateSlow();
+    ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+    assertNotNull(ref);
+    assertFalse(ref.isLazilyLoaded());
+    assertEquals(0, ref.get().getZNodeVersion());
+
+    // update the collection
+    state =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            ref.get().getZNodeVersion());
+    wc = new ZkWriteCommand("c1", state);
+    writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+    writer.writePendingUpdates();
+
+    reader.forciblyRefreshAllClusterStateSlow();
+    ref = reader.getClusterState().getCollectionRef("c1");
+    assertNotNull(ref);
+    assertFalse(ref.isLazilyLoaded());
+    assertEquals(1, ref.get().getZNodeVersion());
+
+    // delete the collection c1, add a collection c2 that is NOT watched
+    ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+    state =
+        new DocCollection(
+            "c2",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+    writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+    writer.writePendingUpdates();
+
+    reader.forciblyRefreshAllClusterStateSlow();
+    ref = reader.getClusterState().getCollectionRef("c1");
+    assertNull(ref);
+
+    ref = reader.getClusterState().getCollectionRef("c2");
+    assertNotNull(ref);
+    assertTrue(
+        "c2 should have been lazily loaded but is not!",
+        ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+    assertEquals(0, ref.get().getZNodeVersion());
+  }
 
-      // create new collection
-      DocCollection state =
-          new DocCollection(
-              "c1",
-              new HashMap<>(),
-              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
-              DocRouter.DEFAULT,
-              0);
-      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
-      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
-      writer.writePendingUpdates();
+  public void testGetCurrentCollections() throws Exception {
+    ZkStateWriter writer = fixture.writer;
+    ZkStateReader reader = fixture.reader;
+
+    reader.registerCore("c1"); // listen to c1. not yet exist
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+    reader.forceUpdateCollection("c1");
+    Set<String> currentCollections = reader.getCurrentCollections();
+    assertEquals(0, currentCollections.size()); // no active collections yet
+
+    // now create both c1 (watched) and c2 (not watched)
+    DocCollection state1 =
+        new DocCollection(
+            "c1",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+    ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+    DocCollection state2 =
+        new DocCollection(
+            "c2",
+            new HashMap<>(),
+            Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+            DocRouter.DEFAULT,
+            0);
+
+    // do not listen to c2
+    fixture.zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+    ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+    writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+    writer.writePendingUpdates();
+
+    reader.forceUpdateCollection("c1");
+    reader.forceUpdateCollection("c2");
+    currentCollections =
+        reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+    // loaded)

Review Comment:
   nit: this is strange wrapping, separate the comment onto its own line?



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -539,17 +672,17 @@ private void constructState(Set<String> changedCollections) {
     if (log.isDebugEnabled()) {
       log.debug(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
-          collectionWatches.keySet().size(),
-          watchedCollectionStates.keySet().size(),
+          collectionWatches.watchedCollections().size(),
+          collectionWatches.activeCollectionCount(),
           lazyCollectionStates.keySet().size(),
           clusterState.getCollectionStates().size());
     }
 
     if (log.isTraceEnabled()) {
       log.trace(
           "clusterStateSet: interesting [{}] watched [{}] lazy [{}] total [{}]",
-          collectionWatches.keySet(),
-          watchedCollectionStates.keySet(),
+          collectionWatches.watchedCollections(),
+          collectionWatches.activeCollections(),
           lazyCollectionStates.keySet(),
           clusterState.getCollectionStates());
     }

Review Comment:
   Is there still technically a race condition here where the watched collections and active collections could change between the successive calls?



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +246,134 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch = statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>> watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether the state has changed for the watched collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      AtomicBoolean stateHasChanged = new AtomicBoolean(false);
+      statefulWatchesByCollectionName.computeIfPresent(
+          collection,
+          (col, watch) -> {
+            DocCollection oldState = watch.currentState;
+            if (oldState == null && newState == null) {
+              // OK, the collection not yet exist in ZK or already deleted
+            } else if (oldState == null) {
+              if (log.isDebugEnabled()) {
+                log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+              }
+              watch.currentState = newState;
+            } else if (newState == null) {
+              log.debug("Removing cached collection state for [{}]", collection);
+              watch.currentState = null;
+            } else { // both new and old states are non-null
+              int oldCVersion =
+                  oldState.getPerReplicaStates() == null
+                      ? -1
+                      : oldState.getPerReplicaStates().cversion;
+              int newCVersion =
+                  newState.getPerReplicaStates() == null
+                      ? -1
+                      : newState.getPerReplicaStates().cversion;
+              if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                  || oldCVersion < newCVersion) {
+                watch.currentState = newState;
+                if (log.isDebugEnabled()) {
+                  log.debug(
+                      "Updating data for [{}] from [{}] to [{}]",
+                      collection,
+                      oldState.getZNodeVersion(),
+                      newState.getZNodeVersion());
+                }
+              }
+            }
+            stateHasChanged.set(oldState != watch.currentState);
+            return watch;
+          });
+
+      return stateHasChanged.get();
+    }
+
+    /**
+     * Computes the new StatefulCollectionWatch by the supplied remappingFunction.

Review Comment:
   nit: add a `@see` javadoc pointing to Map.compute



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -525,8 +652,14 @@ private void constructState(Set<String> changedCollections) {
     Map<String, ClusterState.CollectionRef> result = new LinkedHashMap<>();
 
     // Add collections
-    for (Map.Entry<String, DocCollection> entry : watchedCollectionStates.entrySet()) {
-      result.put(entry.getKey(), new ClusterState.CollectionRef(entry.getValue()));
+    for (Entry<String, StatefulCollectionWatch> entry :
+        collectionWatches.watchedCollectionEntries()) {
+      if (entry.getValue().currentState != null) {
+        // if the doc is null for the collection watch, then it should not be inserted into the
+        // state
+        result.putIfAbsent(

Review Comment:
   do we need to check the result here? what does it mean if there was already an entry and this call fails?



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +432,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.watchedCollections()) {
         DocCollection newState = fetchCollectionState(coll, null);

Review Comment:
   Please add this justification as a comment to the code.



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +246,134 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches {
+    private final ConcurrentHashMap<String, StatefulCollectionWatch>
+        statefulWatchesByCollectionName = new ConcurrentHashMap<>();
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      StatefulCollectionWatch watch = statefulWatchesByCollectionName.get(collection);
+      return watch != null ? watch.currentState : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Gets the count of active collections (collections that exist) being watched
+     *
+     * @return the count of active collections
+     */
+    private long activeCollectionCount() {
+      return statefulWatchesByCollectionName.entrySet().stream()
+          .filter(
+              (Entry<String, StatefulCollectionWatch> entry) ->
+                  entry.getValue().currentState != null)
+          .count();
+    }
+
+    private Set<String> watchedCollections() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.keySet());
+    }
+
+    private Set<Entry<String, StatefulCollectionWatch>> watchedCollectionEntries() {
+      return Collections.unmodifiableSet(statefulWatchesByCollectionName.entrySet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link StatefulCollectionWatch} if
+     * the collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether the state has changed for the watched collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      AtomicBoolean stateHasChanged = new AtomicBoolean(false);
+      statefulWatchesByCollectionName.computeIfPresent(
+          collection,
+          (col, watch) -> {
+            DocCollection oldState = watch.currentState;
+            if (oldState == null && newState == null) {
+              // OK, the collection not yet exist in ZK or already deleted
+            } else if (oldState == null) {
+              if (log.isDebugEnabled()) {
+                log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+              }
+              watch.currentState = newState;
+            } else if (newState == null) {
+              log.debug("Removing cached collection state for [{}]", collection);
+              watch.currentState = null;
+            } else { // both new and old states are non-null
+              int oldCVersion =
+                  oldState.getPerReplicaStates() == null
+                      ? -1
+                      : oldState.getPerReplicaStates().cversion;
+              int newCVersion =
+                  newState.getPerReplicaStates() == null
+                      ? -1
+                      : newState.getPerReplicaStates().cversion;
+              if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+                  || oldCVersion < newCVersion) {

Review Comment:
   Can we add a test that creates c1, gets the ref, deletes c1, creates c1 again, and then checks that the new ref is correct? The node version for both instances of c1 would be 0, but the cversion should be larger, so I think your logic still works, but I'd like to see a unit test for it. Or maybe this is already covered and I missed where it is in the tests? Thanks! Similar to what we saw in SOLR-16143.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] cpoerschke commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
cpoerschke commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1184769765

   > ... could you check-off the `Allow maintainers to commit` option?
   
   It seems that the PR might be from an organisation repo and possibly the https://github.com/isaacs/github/issues/1681 might still apply then, just FYI in case it does.
   
   


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1190429359

   Actually I can't push to your PR.
   
   Can you add this at the end of the 9.1.0 "Improvements" section:
   
   ```
   * SOLR-16257: Improve ZkStateReader to avoid race condition between collectionWatches and watchedCollectionStates
     (Patson Luk, Houston Putman, Mike Drob)
   ```


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1159209423

   The details of one test failure that I cannot reproduce:
   
   ```
     2> 1444230 INFO  (SUITE-TestTlogReplica-seed#[C004E5CD6B2D226F]-worker) [n:127.0.0.1:56946_solr] o.a.s.c.ZkTestServer Shutting down ZkTestServer.
     2> 1444438 WARN  (ZkTestServer Run Thread) [] o.a.s.c.ZkTestServer Watch limit violations: 
     2> Maximum concurrent create/delete watches above limit:
     2> 
     2> 	33	/solr/collections/tlog_replica_test_create_delete/terms/shard1
     2> 	33	/solr/configs/conf
     2> 	31	/solr/collections/tlog_replica_test_create_delete/terms/shard2
     2> 	20	/solr/collections/tlog_replica_test_remove_leader/terms/shard1
     2> 	20	/solr/collections/tlog_replica_test_create_delete/state.json
     2> 	19	/solr/collections/tlog_replica_test_recovery/terms/shard1
     2> 	18	/solr/collections/tlog_replica_test_real_time_get/terms/shard1
     2> 	17	/solr/collections/tlog_replica_test_basic_leader_election/terms/shard1
     2> 	17	/solr/collections/tlog_replica_test_kill_leader/terms/shard1
     2> 	15	/solr/collections/tlog_replica_test_add_docs/terms/shard1
     2> 	11	/solr/collections/tlog_replica_test_delete_by_id/terms/shard1
     2> 	11	/solr/collections/tlog_replica_test_only_leader_indexes/terms/shard1
     2> 	10	/solr/collections/tlog_replica_test_rebalance_leaders/terms/shard1
     2> 	9	/solr/collections/tlog_replica_test_out_of_order_db_qwith_in_place_updates/terms/shard1
     2> 	9	/solr/aliases.json
     2> 	9	/solr/clusterprops.json
     2> 	8	/solr/packages.json
     2> 	8	/solr/security.json
     2> 	8	/solr/collections/tlog_replica_test_add_remove_tlog_replica/terms/shard1
     2> 	8	/solr/collections/tlog_replica_test_add_remove_tlog_replica/terms/shard2
     2> 	8	/solr/collections/tlog_replica_test_real_time_get/state.json
     2> 	7	/solr/collections/tlog_replica_test_add_docs/state.json
     2> 	7	/solr/collections/tlog_replica_test_remove_leader/state.json
     2> 	7	/solr/collections/tlog_replica_test_add_remove_tlog_replica/state.json
     2> 	6	/solr/collections/tlog_replica_test_basic_leader_election/state.json
     2> 	6	/solr/collections/tlog_replica_test_only_leader_indexes/state.json
     2> 	5	/solr/collections/tlog_replica_test_out_of_order_db_qwith_in_place_updates/state.json
     2> 	5	/solr/collections/tlog_replica_test_kill_leader/state.json
     2> 	5	/solr/collections/tlog_replica_test_recovery/collectionprops.json
     2> 	5	/solr/collections/tlog_replica_test_rebalance_leaders/state.json
     2> 	5	/solr/collections/tlog_replica_test_delete_by_id/state.json
     2> 	4	/solr/collections/tlog_replica_test_recovery/state.json
     2> 	3	/solr/collections/tlog_replica_test_rebalance_leaders/collectionprops.json
     2> 	3	/solr/collections/tlog_replica_test_kill_leader/collectionprops.json
     2> 	3	/solr/collections/tlog_replica_test_basic_leader_election/collectionprops.json
     2> 	2	/solr/collections/tlog_replica_test_delete_by_id/collectionprops.json
     2> 	2	/solr/collections/tlog_replica_test_add_remove_tlog_replica/collectionprops.json
     2> 	2	/solr/collections/tlog_replica_test_out_of_order_db_qwith_in_place_updates/collectionprops.json
     2> 	2	/solr/collections/tlog_replica_test_add_docs/collectionprops.json
     2> 	2	/solr/collections/tlog_replica_test_remove_leader/collectionprops.json
     2> 	2	/solr/collections/tlog_replica_test_only_leader_indexes/collectionprops.json
     2> 	2	/solr/collections/tlog_replica_test_real_time_get/collectionprops.json
     2> 
     2> Maximum concurrent data watches above limit:
     2> 
     2> 	97	/solr/collections/tlog_replica_test_recovery/state.json
     2> 	79	/solr/collections/tlog_replica_test_create_delete/state.json
     2> 	51	/solr/collections/tlog_replica_test_kill_leader/state.json
     2> 	49	/solr/collections/tlog_replica_test_out_of_order_db_qwith_in_place_updates/state.json
     2> 	48	/solr/collections/tlog_replica_test_basic_leader_election/state.json
     2> 	44	/solr/collections/tlog_replica_test_remove_leader/state.json
     2> 	37	/solr/collections/tlog_replica_test_add_remove_tlog_replica/state.json
     2> 	21	/solr/collections/tlog_replica_test_rebalance_leaders/state.json
     2> 	20	/solr/collections/tlog_replica_test_real_time_get/state.json
     2> 	19	/solr/collections/tlog_replica_test_add_docs/state.json
     2> 	15	/solr/collections/tlog_replica_test_only_leader_indexes/state.json
     2> 	15	/solr/collections/tlog_replica_test_delete_by_id/state.json
     2> 	4	/solr/collections/tlog_replica_test_recovery/leader_elect/shard1/election/72082096014163974-core_node4-n_0000000000
     2> 	4	/solr/overseer_elect/election/72082096014163974-127.0.0.1:56945_solr-n_0000000001
     2> 	3	/solr/collections/tlog_replica_test_create_delete/leader_elect/shard1/election/72082096014163984-core_node11-n_0000000000
     2> 	2	/solr/collections/tlog_replica_test_create_delete/leader_elect/shard1/election/72082096014163984-core_node12-n_0000000000
     2> 	2	/solr/overseer_elect/election/72082096014163984-127.0.0.1:56946_solr-n_0000000005
     2> 	2	/solr/collections/tlog_replica_test_create_delete/leader_elect/shard2/election/72082096014163984-core_node9-n_0000000001
     2> 	2	/solr/collections/tlog_replica_test_create_delete/leader_elect/shard1/election/72082096014163988-core_node14-n_0000000001
     2> 
     2> Maximum concurrent children watches above limit:
     2> 
     2> 	87	/solr/collections
     2> 	52	/solr/live_nodes
     2> 	48	/solr/collections/tlog_replica_test_create_delete/state.json
     2> 	32	/solr/collections/tlog_replica_test_recovery/state.json
     2> 	22	/solr/collections/tlog_replica_test_kill_leader/state.json
     2> 	21	/solr/collections/tlog_replica_test_out_of_order_db_qwith_in_place_updates/state.json
     2> 	20	/solr/collections/tlog_replica_test_basic_leader_election/state.json
     2> 	20	/solr/collections/tlog_replica_test_remove_leader/state.json
     2> 	14	/solr/collections/tlog_replica_test_add_docs/state.json
     2> 	13	/solr/collections/tlog_replica_test_add_remove_tlog_replica/state.json
     2> 	13	/solr/collections/tlog_replica_test_real_time_get/state.json
     2> 	12	/solr/collections/tlog_replica_test_rebalance_leaders/state.json
     2> 	12	/solr/collections/tlog_replica_test_only_leader_indexes/state.json
     2> 	12	/solr/collections/tlog_replica_test_delete_by_id/state.json
     2> 	3	/solr/overseer/queue
     2> 	3	/solr/overseer/collection-queue-work
     2> 	3	/solr/overseer/queue-work
     2> 
   ```


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] HoustonPutman commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
HoustonPutman commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r903111897


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -525,8 +606,14 @@ private void constructState(Set<String> changedCollections) {
     Map<String, ClusterState.CollectionRef> result = new LinkedHashMap<>();
 
     // Add collections
-    for (Map.Entry<String, DocCollection> entry : watchedCollectionStates.entrySet()) {
-      result.put(entry.getKey(), new ClusterState.CollectionRef(entry.getValue()));
+    for (Map.Entry<String, DocCollectionWatch<DocCollectionWatcher>> entry :
+        collectionWatches.entrySet()) {
+      if (entry.getValue().currentDoc
+          != null) { // if the doc is null for the collection watch, then it should not be inserted

Review Comment:
   Can you put the comment on the next line? That way the formatting of the if statement is preserved.



##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -16,28 +16,32 @@
  */
 package org.apache.solr.cloud.overseer;
 
+import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.*;

Review Comment:
   This might be an IDE thing, but please don't compress these. (Also happened below)



##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -246,6 +244,90 @@ public boolean canBeRemoved() {
     }
   }
 
+  /**
+   * A ConcurrentHashMap of active watcher by collection name
+   *
+   * <p>Each watcher DocCollectionWatch also contains the latest DocCollection (state) observed
+   */
+  private static class DocCollectionWatches
+      extends ConcurrentHashMap<String, DocCollectionWatch<DocCollectionWatcher>> {
+
+    /**
+     * Gets the DocCollection (state) of the collection which the corresponding watch last observed
+     *
+     * @param collection the collection name to get DocCollection on
+     * @return The last observed DocCollection(state). if null, that means there's no such
+     *     collection.
+     */
+    private DocCollection getDocCollection(String collection) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      return watch != null ? watch.currentDoc : null;
+    }
+
+    /**
+     * Gets the active collections (collections that exist) being watched
+     *
+     * @return an immutable set of active collection names
+     */
+    private Set<String> activeCollections() {
+      return this.entrySet().stream()
+          .filter(
+              (Entry<String, DocCollectionWatch<DocCollectionWatcher>> entry) ->
+                  entry.getValue().currentDoc != null)
+          .map(Entry::getKey)
+          .collect(Collectors.toUnmodifiableSet());
+    }
+
+    /**
+     * Updates the latest observed DocCollection (state) of the {@link DocCollectionWatch} if the
+     * collection is being watched
+     *
+     * @param collection the collection name
+     * @param newState the new DocCollection (state) observed
+     * @return whether an active watch exists for such collection
+     */
+    private boolean updateDocCollection(String collection, DocCollection newState) {
+      DocCollectionWatch<DocCollectionWatcher> watch = get(collection);
+      if (watch != null) {
+        DocCollection oldState = watch.currentDoc;
+        if (oldState == null && newState == null) {
+          // OK, the collection not yet exist in ZK
+        } else if (oldState == null) {
+          if (log.isDebugEnabled()) {
+            log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
+          }
+          watch.currentDoc = newState;
+        } else if (newState == null) {
+          log.debug("Removing cached collection state for [{}]", collection);
+          watch.currentDoc = null;
+        } else { // both new and old states are non-null
+          int oldCVersion =
+              oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
+          int newCVersion =
+              newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
+          if (oldState.getZNodeVersion() < newState.getZNodeVersion()
+              || oldCVersion < newCVersion) {
+            watch.currentDoc = newState;
+            if (log.isDebugEnabled()) {
+              log.debug(
+                  "Updating data for [{}] from [{}] to [{}]",
+                  collection,
+                  oldState.getZNodeVersion(),
+                  newState.getZNodeVersion());
+            }
+          }
+        }
+        return true;
+      } else {
+        return false;
+      }
+    }

Review Comment:
   It looks like this is where the race condition could occur. There's no guarantee that `updateDocCollection()` is not called at the same time for the same collection. However, it looks like this race condition could still occur.
   
   ```suggestion
       DocCollectionWatch<DocCollectionWatcher> finalWatch = computeIfPresent(collection, (col, watch) -> {
         if (watch != null) {
           DocCollection oldState = watch.currentDoc;
           if (oldState == null && newState == null) {
             // OK, the collection not yet exist in ZK
           } else if (oldState == null) {
             if (log.isDebugEnabled()) {
               log.debug("Add data for [{}] ver [{}]", collection, newState.getZNodeVersion());
             }
             watch.currentDoc = newState;
           } else if (newState == null) {
             log.debug("Removing cached collection state for [{}]", collection);
             watch.currentDoc = null;
           } else { // both new and old states are non-null
             int oldCVersion =
                 oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
             int newCVersion =
                 newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
             if (oldState.getZNodeVersion() < newState.getZNodeVersion()
                 || oldCVersion < newCVersion) {
               watch.currentDoc = newState;
               if (log.isDebugEnabled()) {
                 log.debug(
                     "Updating data for [{}] from [{}] to [{}]",
                     collection,
                     oldState.getZNodeVersion(),
                     newState.getZNodeVersion());
               }
             }
           }
         }
         return watch;
       });
       return finalWatch != null;
   ```
   
   Also I'm not sure that we need to be creating a new class for this, we can likely just keep a ConcurrentHashMap variable, and have the helper methods outside. But let's focus first on the logic



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on PR #909:
URL: https://github.com/apache/solr/pull/909#issuecomment-1162438905

   The original junit error was 
   ```
    2> 1393351 INFO  (TEST-TestTlogReplica.testRebalanceLeaders-seed#[C004E5CD6B2D226F]) [] o.a.s.SolrTestCaseJ4 ###Ending testRebalanceLeaders
      >     java.lang.AssertionError: Can not find doc 1 in http://127.0.0.1:56945/solr
      >         at __randomizedtesting.SeedInfo.seed([C004E5CD6B2D226F:DEEBD73D53866E1F]:0)
      >         at org.junit.Assert.fail(Assert.java:89)
      >         at org.junit.Assert.assertTrue(Assert.java:42)
      >         at org.junit.Assert.assertNotNull(Assert.java:713)
      >         at org.apache.solr.cloud.TestTlogReplica.checkRTG(TestTlogReplica.java:1138)
      >         at org.apache.solr.cloud.TestTlogReplica.testRebalanceLeaders(TestTlogReplica.java:839)
      >         at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      >         at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
      >         at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   ```
   Seems to not happening anymore after the additional commits


-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r911530388


##########
solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java:
##########
@@ -326,12 +408,11 @@ public void forciblyRefreshAllClusterStateSlow() throws KeeperException, Interru
       // No need to set watchers because we should already have watchers registered for everything.
       refreshCollectionList(null);
       refreshLiveNodes(null);
-      // Need a copy so we don't delete from what we're iterating over.
-      Collection<String> safeCopy = new ArrayList<>(watchedCollectionStates.keySet());
+
       Set<String> updatedCollections = new HashSet<>();
-      for (String coll : safeCopy) {
+      for (String coll : collectionWatches.keySet()) {

Review Comment:
   Ah yes. This is one of the change that Im not 100% sure as it changes the behavior. It was buried in one of the original questions:
   
   1. In `forciblyRefreshAllClusterStateSlow`, which we are proposing to just replace `watchedCollectionStates.keySet()` with `collectionWatches.keySet()`, the reasoning is that we should forcibly update every collection that is registered in `collectionWatches`, even if previous fetch on such collection returned null (ie `watchedCollectionStates` would have no entry on it). This is a minor change of behavior but I think it is more "correct"? Would really need more experienced Solr dev to share their thoughts here 🙏🏼 
   
   So my reasoning is that `forciblyRefreshAllClusterStateSlow` is supposed to "discover" new collections if it was previously watched but not yet existed in last fetch.



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] patsonluk commented on a diff in pull request #909: SOLR-16257: `ZkStateReader` changes to avoid race condition between `collectionWatches` and `watchedCollectionStates`

Posted by GitBox <gi...@apache.org>.
patsonluk commented on code in PR #909:
URL: https://github.com/apache/solr/pull/909#discussion_r911547445


##########
solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java:
##########
@@ -196,4 +209,251 @@ public void testWatchedCollectionCreation() throws Exception {
       server.shutdown();
     }
   }
+
+  public void testForciblyRefreshAllClusterState() throws Exception {
+    Path zkDir = createTempDir("testForciblyRefreshAllClusterState");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // watching c1, so it should get non lazy reference
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      // Initially there should be no c1 collection.
+      assertNull(reader.getClusterState().getCollectionRef("c1"));
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+
+      // create new collection
+      DocCollection state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      assertTrue(zkClient.exists(ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json", true));
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ClusterState.CollectionRef ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(0, ref.get().getZNodeVersion());
+
+      // update the collection
+      state =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              ref.get().getZNodeVersion());
+      wc = new ZkWriteCommand("c1", state);
+      writer.enqueueUpdate(reader.getClusterState(), Collections.singletonList(wc), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNotNull(ref);
+      assertFalse(ref.isLazilyLoaded());
+      assertEquals(1, ref.get().getZNodeVersion());
+
+      // delete the collection c1, add a collection c2 that is NOT watched
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", null);
+
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      state =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state);
+
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forciblyRefreshAllClusterStateSlow();
+      ref = reader.getClusterState().getCollectionRef("c1");
+      assertNull(ref);
+
+      ref = reader.getClusterState().getCollectionRef("c2");
+      assertNotNull(ref);
+      assert (ref.isLazilyLoaded()); // c2 should be lazily loaded as it's not watched
+      assertEquals(0, ref.get().getZNodeVersion());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testGetCurrentCollections() throws Exception {
+    Path zkDir = createTempDir("testGetCurrentCollections");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      reader.createClusterStateWatchersAndUpdate();
+      reader.registerCore("c1"); // listen to c1. not yet exist
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+      reader.forceUpdateCollection("c1");
+      Set<String> currentCollections = reader.getCurrentCollections();
+      assertEquals(0, currentCollections.size()); // no active collections yet
+
+      // now create both c1 (watched) and c2 (not watched)
+      DocCollection state1 =
+          new DocCollection(
+              "c1",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+      ZkWriteCommand wc1 = new ZkWriteCommand("c1", state1);
+      DocCollection state2 =
+          new DocCollection(
+              "c2",
+              new HashMap<>(),
+              Map.of(ZkStateReader.CONFIGNAME_PROP, ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+              DocRouter.DEFAULT,
+              0);
+
+      // do not listen to c2
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c2", true);
+      ZkWriteCommand wc2 = new ZkWriteCommand("c2", state2);
+
+      ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      writer.enqueueUpdate(reader.getClusterState(), Arrays.asList(wc1, wc2), null);
+      writer.writePendingUpdates();
+
+      reader.forceUpdateCollection("c1");
+      reader.forceUpdateCollection("c2");
+      currentCollections =
+          reader.getCurrentCollections(); // should detect both collections (c1 watched, c2 lazy
+      // loaded)
+      assertEquals(2, currentCollections.size());
+    } finally {
+      IOUtils.close(reader, zkClient);
+      server.shutdown();
+    }
+  }
+
+  public void testWatchRaceCondition() throws Exception {
+    final int RUN_COUNT = 10000;
+    Path zkDir = createTempDir("testWatchRaceCondition");
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+
+    SolrZkClient zkClient = null;
+    ZkStateReader reader = null;
+    ExecutorService executorService =
+        ExecutorUtil.newMDCAwareSingleThreadExecutor(
+            new SolrNamedThreadFactory("zkStateReaderTest"));
+
+    try {
+      server.run();
+
+      zkClient = new SolrZkClient(server.getZkAddress(), OverseerTest.DEFAULT_CONNECTION_TIMEOUT);
+      ZkController.createClusterZkNodes(zkClient);
+
+      reader = new ZkStateReader(zkClient);
+      final ZkStateReader readerRef = reader;
+      reader.createClusterStateWatchersAndUpdate();
+      zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/c1", true);
+
+      // start another thread to constantly updating the state
+      final AtomicBoolean stopMutatingThread = new AtomicBoolean(false);
+      final ZkStateWriter writer = new ZkStateWriter(reader, new Stats());
+      final AtomicInteger updateCounts = new AtomicInteger(0);
+      final AtomicReference<Exception> updateException = new AtomicReference<>();
+      executorService.submit(
+          () -> {
+            try {
+              ClusterState clusterState = readerRef.getClusterState();
+              while (!stopMutatingThread.get()) {
+                DocCollection collection = clusterState.getCollectionOrNull("c1");
+                int currentVersion = collection != null ? collection.getZNodeVersion() : 0;
+                // create new collection
+                DocCollection state =
+                    new DocCollection(
+                        "c1",
+                        new HashMap<>(),
+                        Map.of(
+                            ZkStateReader.CONFIGNAME_PROP,
+                            ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
+                        DocRouter.DEFAULT,
+                        currentVersion);
+                ZkWriteCommand wc = new ZkWriteCommand("c1", state);
+                writer.enqueueUpdate(clusterState, Collections.singletonList(wc), null);
+                clusterState = writer.writePendingUpdates();
+              }
+            } catch (Exception e) {
+              updateException.set(e);
+            }
+            return null;
+          });
+      executorService.shutdown();
+
+      for (int i = 0; i < RUN_COUNT; i++) {
+        final CountDownLatch latch = new CountDownLatch(2);
+
+        // remove itself on 2nd trigger
+        DocCollectionWatcher dummyWatcher =
+            collection -> {
+              latch.countDown();
+              return latch.getCount() == 0;
+            };
+        reader.registerDocCollectionWatcher("c1", dummyWatcher);
+        latch.await(10, TimeUnit.SECONDS);
+        reader.removeDocCollectionWatcher("c1", dummyWatcher);
+
+        boolean refLazilyLoaded = false;
+        for (int j = 0; j < 10; j++) {
+          if (reader.getClusterState().getCollectionRef("c1").isLazilyLoaded()) {
+            refLazilyLoaded = true; // it should eventually be lazily loaded
+            break;
+          }
+          int attempt = j + 1;
+          log.info("ref is not lazily loaded yet. Attempt : {}", attempt);
+
+          TimeUnit.MILLISECONDS.sleep(100);
+        }
+        assert (refLazilyLoaded);

Review Comment:
   By using TimeOut.waitFor, I can remove such check :)



-- 
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: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org