You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/02/28 00:04:23 UTC

[GitHub] [lucene-solr] tflobbe commented on a change in pull request #1297: SOLR-14253 Replace various sleep calls with ZK waits

tflobbe commented on a change in pull request #1297: SOLR-14253 Replace various sleep calls with ZK waits
URL: https://github.com/apache/lucene-solr/pull/1297#discussion_r385438210
 
 

 ##########
 File path: solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
 ##########
 @@ -526,57 +518,31 @@ static UpdateResponse softCommit(String url) throws SolrServerException, IOExcep
   }
 
   String waitForCoreNodeName(String collectionName, String msgNodeName, String msgCore) {
-    int retryCount = 320;
-    while (retryCount-- > 0) {
-      final DocCollection docCollection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
-      if (docCollection != null && docCollection.getSlicesMap() != null) {
-        Map<String,Slice> slicesMap = docCollection.getSlicesMap();
-        for (Slice slice : slicesMap.values()) {
-          for (Replica replica : slice.getReplicas()) {
-            // TODO: for really large clusters, we could 'index' on this
-
-            String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
-            String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
-
-            if (nodeName.equals(msgNodeName) && core.equals(msgCore)) {
-              return replica.getName();
-            }
-          }
+    AtomicReference<String> coreNodeName = new AtomicReference<>();
+    try {
+      zkStateReader.waitForState(collectionName, 320, TimeUnit.SECONDS, c -> {
+        String name = ClusterStateMutator.getAssignedCoreNodeName(c, msgNodeName, msgCore);
+        if (name == null) {
+          return false;
         }
-      }
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      }
+        coreNodeName.set(name);
+        return true;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Timeout waiting for collection state", e);
     }
-    throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find coreNodeName");
+    return coreNodeName.get();
   }
 
-  ClusterState waitForNewShard(String collectionName, String sliceName) throws KeeperException, InterruptedException {
+  ClusterState waitForNewShard(String collectionName, String sliceName) {
     log.debug("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
-    RTimer timer = new RTimer();
-    int retryCount = 320;
-    while (retryCount-- > 0) {
-      ClusterState clusterState = zkStateReader.getClusterState();
-      DocCollection collection = clusterState.getCollection(collectionName);
-
-      if (collection == null) {
-        throw new SolrException(ErrorCode.SERVER_ERROR,
-            "Unable to find collection: " + collectionName + " in clusterstate");
-      }
-      Slice slice = collection.getSlice(sliceName);
-      if (slice != null) {
-        log.debug("Waited for {}ms for slice {} of collection {} to be available",
-            timer.getTime(), sliceName, collectionName);
-        return clusterState;
-      }
-      Thread.sleep(1000);
+    try {
+      zkStateReader.waitForState(collectionName, 320, TimeUnit.SECONDS, c -> c != null && c.getSlice(sliceName) != null);
+    } catch (TimeoutException | InterruptedException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Timeout waiting for new slice", e);
     }
-    throw new SolrException(ErrorCode.SERVER_ERROR,
-        "Could not find new slice " + sliceName + " in collection " + collectionName
-            + " even after waiting for " + timer.getTime() + "ms"
-    );
+    // nocommit is there a race condition here since we're not returning the same clusterstate we inspected?
 
 Review comment:
   Isn't that the case with most of this methods? While the predicate is being executed for example, there is no watch in ZooKeeper AFAICT, unless we go back and write in ZooKeeper and use the version.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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