murblanc commented on a change in pull request #2318:
URL: https://github.com/apache/lucene-solr/pull/2318#discussion_r573680423



##########
File path: 
solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
##########
@@ -264,8 +299,8 @@ public void call(ClusterState clusterState, ZkNodeProps 
message, @SuppressWarnin
         log.info("Cleaned up artifacts for failed create collection for [{}]", 
collectionName);
         throw new SolrException(ErrorCode.BAD_REQUEST, "Underlying core 
creation failed while creating collection: " + collectionName);
       } else {
-        log.debug("Finished create command on all shards for collection: {}", 
collectionName);
 
+        log.debug("Finished create command on all shards for collection: {}", 
collectionName);

Review comment:
       When we get here for a PRS collection, the `ZkStateReader` cluster state 
for the collection is fine, as are the ZK structures, but the Overseer cluster 
state updater does not know about the collection. Any new operation on the 
collection through the Collection API would fail in the cluster state updater.

##########
File path: 
solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
##########
@@ -149,27 +143,38 @@ public void call(ClusterState clusterState, ZkNodeProps 
message, @SuppressWarnin
       }
 
       createCollectionZkNode(stateManager, collectionName, collectionParams);
-      
-      ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
-
-      // wait for a while until we see the collection
-      TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS, timeSource);
-      boolean created = false;
-      while (! waitUntil.hasTimedOut()) {
-        waitUntil.sleep(100);
-        created = 
ocmh.cloudManager.getClusterStateProvider().getClusterState().hasCollection(collectionName);
-        if(created) break;
-      }
-      if (!created) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could 
not fully create collection: " + collectionName);
+
+      if(isPrs) {
+        ZkWriteCommand command = new 
ClusterStateMutator(ocmh.cloudManager).createCollection(clusterState, message);
+        byte[] data = Utils.toJSON(Collections.singletonMap(collectionName, 
command.collection));
+        ocmh.zkStateReader.getZkClient().create(collectionPath, data, 
CreateMode.PERSISTENT, true);
+        clusterState = clusterState.copyWith(collectionName, 
command.collection);
+        newColl = command.collection;
+      } else {
+        ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
+
+        // wait for a while until we see the collection
+        TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS, timeSource);
+        boolean created = false;
+        while (!waitUntil.hasTimedOut()) {
+          waitUntil.sleep(100);
+          created = 
ocmh.cloudManager.getClusterStateProvider().getClusterState().hasCollection(collectionName);
+          if (created) break;
+        }
+        if (!created) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could 
not fully create collection: " + collectionName);
+        }
+
+        // refresh cluster state
+        clusterState = 
ocmh.cloudManager.getClusterStateProvider().getClusterState();

Review comment:
       This call does nothing and can be removed.
   The command already has the instance of `ClusterState` returned  by the call 
(it is passed to it as a `call` parameter by OCMH).
   The only way the clusterState is refreshed is when the watchers on the node 
get called and fetch the latest updates from ZK (that in this case were done by 
the Overseer, and we know the watchers fired and updated our local state 
because we've waited for it in the `while` loop ~10 lines above).

##########
File path: 
solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
##########
@@ -149,27 +143,38 @@ public void call(ClusterState clusterState, ZkNodeProps 
message, @SuppressWarnin
       }
 
       createCollectionZkNode(stateManager, collectionName, collectionParams);
-      
-      ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
-
-      // wait for a while until we see the collection
-      TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS, timeSource);
-      boolean created = false;
-      while (! waitUntil.hasTimedOut()) {
-        waitUntil.sleep(100);
-        created = 
ocmh.cloudManager.getClusterStateProvider().getClusterState().hasCollection(collectionName);
-        if(created) break;
-      }
-      if (!created) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could 
not fully create collection: " + collectionName);
+
+      if(isPrs) {
+        ZkWriteCommand command = new 
ClusterStateMutator(ocmh.cloudManager).createCollection(clusterState, message);

Review comment:
       Calling this method here (or more specifically using its output in the 
`create()` call below) breaks the cluster state updater contract in Overseer.
   This call updates a `state.json` from a thread other than the cluster state 
updater thread in Overseer, the cluster state updater will not know about the 
new Json being written (the fact that this thread running this method is 
executing on the Overseer node is irrelevant, except passing stats, cluster 
state updater and Collection API execution are totally independent and could be 
running on different nodes).
   
   Overseer has two copies of the cluster state: the cluster state maintained 
in `ZkStateReader` like any node has (this is the state passed to Collection 
API calls), and another copy of the state maintained in `ZkStateWriter` which 
initially is obtained from `ZkStateReader` but quickly diverges and then used 
as the "source of truth" by the cluster state updater that keeps writing back 
that state to Zookeeper for the whole life of the Overseer on the node. The 
cluster updater state has no watches is only updated by the cluster state 
updater when processing messages and then the updated value written to ZK.
   
   To achieve the effect of dealing with the Per Replica States directly from 
this thread rather than from the ClusterStateUpdater single thread, the 
state.json would have to be created by the Overseer (i.e. through a classic 
`ocmh.overseer.offerStateUpdate(Utils.toJSON(message));`) and once done the PRS 
could be created from here. This is complicated to do because of the way 
`ZkStateWriter` is implemented to write the `state.json` as well as handle the 
PRS operations in the same method (when it calls `cmd.ops.persist` in 
`writePendingUpdates()`) and would require quite some refactoring.

##########
File path: 
solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
##########
@@ -213,7 +218,16 @@ public void call(ClusterState clusterState, ZkNodeProps 
message, @SuppressWarnin
             ZkStateReader.NODE_NAME_PROP, nodeName,
             ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
             CommonAdminParams.WAIT_FOR_FINAL_STATE, 
Boolean.toString(waitForFinalState));
-        ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
+        if(isPrs) {
+          ZkWriteCommand command = new 
SliceMutator(ocmh.cloudManager).addReplica(clusterState, props);
+          byte[] data = Utils.toJSON(Collections.singletonMap(collectionName, 
command.collection));
+//        log.info("collection updated : {}", new String(data, 
StandardCharsets.UTF_8));
+          ocmh.zkStateReader.getZkClient().setData(collectionPath, data, true);

Review comment:
       This call bypasses the cluster state updater to update the `state.json`. 
If it tried to do so through the Overseer cluster state updater, the update 
would fail as the collection is unknown there.




----------------------------------------------------------------
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:
us...@infra.apache.org



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

Reply via email to