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/05/18 22:47:41 UTC

[GitHub] [lucene-solr] murblanc opened a new pull request #1528: SOLR-12823: remove /clusterstate.json

murblanc opened a new pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528


   Remove all code dealing with Zookeeper's /clusterstate.json, remove Collection API's MIGRATESTATEVERSION, remove legacyCloud option.
   
   Notes:
   - org.apache.solr.cloud.autoscaling.sim is non functional, requires more work.
   - TestZkChroot requires more work
   - BasicZkTest requires more work (or better: be deleted)
   
   Also fixes SOLR-11877: DocCollection.getStateFormat is buggy
   
   # Description
   
   Remove all code dealing with /clusterstate.json in Zookeeper, remove MIGRATESTATEVERSION Collection API action and the notion of “legacyCloud”.
   
   Note this is a draft PR at this stage, need to fix the autoscale sim before it can be merged.
   
   # Solution
   
   Solr 9 will refuse to start if an existing /clusterstate.json in Zookeeper exists and is not empty.
   
   # Tests
   
   No new tests, but adapting old tests that were misbehaving.
   
   # 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.
   - [X] 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 `master` branch.
   - [X] I have run `ant precommit` and the appropriate test suite.
   - [ ] I have added tests for my changes.
   - [X] I have added documentation for the [Ref Guide](https://github.com/apache/lucene-solr/tree/master/solr/solr-ref-guide) (for Solr changes only).
   


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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r427627276



##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +493,40 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (data.length < 5) {
+        // less than 5 chars is empty (it's likely just "{}"). This log will only occur once.
+        log.warn("{} no longer supported starting with Solr 9. Found empty file on Zookeeper, deleting it.", ZkStateReader.UNSUPPORTED_CLUSTER_STATE);
+        zkClient.delete(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, -1, true);
+      } else {
+        // /clusterstate.json not empty: refuse to start but do not automatically delete. A bit of a pain but user shouldn't
+        // have older collections at this stage anyway.
+        String message = ZkStateReader.UNSUPPORTED_CLUSTER_STATE + " no longer supported starting with Solr 9. "
+            + "It is present and not empty. Cannot start Solr. Please first migrate collections to stateFormat=2 using an "
+            + "older version of Solr or if you don't care about the data then delete the file from "
+            + "Zookeeper using a command line tool, for example: bin/solr zk rm /clusterstate.json -z host:port";
+        log.error(message);
+        throw new SolrException(SolrException.ErrorCode.INVALID_STATE, message);
+      }
+    } catch (KeeperException e) {
+      log.error("", e);

Review comment:
       Converting checked to unchecked exception. copied from init() further down that deals with the same problem on access to ZK.




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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r427655970



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
##########
@@ -468,36 +467,6 @@ void checkResults(String label, NamedList<Object> results, boolean failureIsFata
     }
   }
 
-
-  //TODO should we not remove in the next release ?
-  private void migrateStateFormat(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {

Review comment:
       Had to remove 1,660 lines and add 568 to get rid of that comment :)




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



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


[GitHub] [lucene-solr] janhoy commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
janhoy commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429830491



##########
File path: solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
##########
@@ -181,16 +180,21 @@ public void close() {
       zkStateReader.close();
     }
 
+    /**
+     * Create a collection.
+     * Note there's a similar but slightly different {@link OverseerTest#createCollection(String, int)}.

Review comment:
       Ok. If/when the big Overseer / Curator rewrite happens, then I guess it will be cleaned up then...




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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r427636430



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
##########
@@ -236,21 +235,19 @@ public void call(ClusterState clusterState, ZkNodeProps message, NamedList resul
         }
 
         String baseUrl = zkStateReader.getBaseUrlForNodeName(nodeName);
-        //in the new mode, create the replica in clusterstate prior to creating the core.
+        //create the replica in clusterstate (i.e. ZK) prior to creating the core.

Review comment:
       Agreed, we should rename in many places. Although here state.json is appropriate since that's what happens.




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



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


[GitHub] [lucene-solr] madrob commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r430596540



##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +493,40 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (data.length < 5) {
+        // less than 5 chars is empty (it's likely just "{}"). This log will only occur once.
+        log.warn("{} no longer supported starting with Solr 9. Found empty file on Zookeeper, deleting it.", ZkStateReader.UNSUPPORTED_CLUSTER_STATE);
+        zkClient.delete(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, -1, true);
+      } else {
+        // /clusterstate.json not empty: refuse to start but do not automatically delete. A bit of a pain but user shouldn't
+        // have older collections at this stage anyway.
+        String message = ZkStateReader.UNSUPPORTED_CLUSTER_STATE + " no longer supported starting with Solr 9. "
+            + "It is present and not empty. Cannot start Solr. Please first migrate collections to stateFormat=2 using an "
+            + "older version of Solr or if you don't care about the data then delete the file from "
+            + "Zookeeper using a command line tool, for example: bin/solr zk rm /clusterstate.json -z host:port";
+        log.error(message);
+        throw new SolrException(SolrException.ErrorCode.INVALID_STATE, message);
+      }
+    } catch (KeeperException e) {
+      log.error("", e);

Review comment:
       Makes sense. If we throw anything here it will still propagate up to SolrDispatchFilter where it will be logged, so we don't need to double up on that here. We should probably clean that up in the other places where it happens, but that's out of scope for 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.

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



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


[GitHub] [lucene-solr] janhoy commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
janhoy commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r427627240



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
##########
@@ -236,21 +235,19 @@ public void call(ClusterState clusterState, ZkNodeProps message, NamedList resul
         }
 
         String baseUrl = zkStateReader.getBaseUrlForNodeName(nodeName);
-        //in the new mode, create the replica in clusterstate prior to creating the core.
+        //create the replica in clusterstate (i.e. ZK) prior to creating the core.

Review comment:
       I think we still can use 'clusterstate' as a generic term? When I read it in a sentence like this I do not read it as the file clusterstate.json, but as the cluster state. Perhaps we could consistenly write it as "cluster state" 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.

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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r427625766



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
##########
@@ -236,21 +235,19 @@ public void call(ClusterState clusterState, ZkNodeProps message, NamedList resul
         }
 
         String baseUrl = zkStateReader.getBaseUrlForNodeName(nodeName);
-        //in the new mode, create the replica in clusterstate prior to creating the core.
+        //create the replica in clusterstate (i.e. ZK) prior to creating the core.

Review comment:
       "clusterstate" is used all over (including in the docs) to refer in general to the state of the cluster or collection stored in ZK.
   Will replace with state.json 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.

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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429817725



##########
File path: solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
##########
@@ -210,47 +200,42 @@ public boolean liveNodesContain(String name) {
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
-    sb.append("znodeVersion: ").append(znodeVersion);
-    sb.append("\n");
     sb.append("live nodes:").append(liveNodes);
     sb.append("\n");
     sb.append("collections:").append(collectionStates);
     return sb.toString();
   }
 
-  public static ClusterState load(Integer version, byte[] bytes, Set<String> liveNodes) {
-    return load(version, bytes, liveNodes, ZkStateReader.CLUSTER_STATE);
-  }
   /**
-   * Create ClusterState from json string that is typically stored in zookeeper.
+   * Create a ClusterState from Json.
    * 
-   * @param version zk version of the clusterstate.json file (bytes)
-   * @param bytes clusterstate.json as a byte array
+   * @param bytes a byte array of a Json representation of a mapping from collection name to the Json representation of a
+   *              {@link DocCollection} as written by {@link #write(JSONWriter)}. It can represent
+   *              one or more collections.
    * @param liveNodes list of live nodes
    * @return the ClusterState
    */
-  public static ClusterState load(Integer version, byte[] bytes, Set<String> liveNodes, String znode) {
-    // System.out.println("######## ClusterState.load:" + (bytes==null ? null : new String(bytes)));
+  public static ClusterState createFromJson(int version, byte[] bytes, Set<String> liveNodes) {
     if (bytes == null || bytes.length == 0) {
-      return new ClusterState(version, liveNodes, Collections.<String, DocCollection>emptyMap());
+      return new ClusterState(liveNodes, Collections.<String, DocCollection>emptyMap());
     }
     Map<String, Object> stateMap = (Map<String, Object>) Utils.fromJSON(bytes);
-    return load(version, stateMap, liveNodes, znode);
+    return createFromData(version, stateMap, liveNodes);
   }
 
-  public static ClusterState load(Integer version, Map<String, Object> stateMap, Set<String> liveNodes, String znode) {
+  public static ClusterState createFromData(int version, Map<String, Object> stateMap, Set<String> liveNodes) {

Review comment:
       As you wish, let me know and I'll change it as I rework the other comments.
   `createFromIntMapAndSet` :)




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



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


[GitHub] [lucene-solr] madrob merged pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
madrob merged pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528


   


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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r427655602



##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +493,40 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (data.length < 5) {

Review comment:
       updated.




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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429817130



##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java
##########
@@ -138,8 +138,7 @@ private ClusterState fetchClusterState(SolrClient client, String collection, Map
     Set<String> liveNodes = new HashSet((List<String>)(cluster.get("live_nodes")));
     this.liveNodes = liveNodes;
     liveNodesTimestamp = System.nanoTime();
-    //TODO SOLR-11877 we don't know the znode path; CLUSTER_STATE is probably wrong leading to bad stateFormat

Review comment:
       Yes, it's mentioned in the PR description.




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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429815739



##########
File path: solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
##########
@@ -181,16 +180,21 @@ public void close() {
       zkStateReader.close();
     }
 
+    /**
+     * Create a collection.
+     * Note there's a similar but slightly different {@link OverseerTest#createCollection(String, int)}.

Review comment:
       I hesitated. There are a few variations in how collections are created. I don't know if these variations are in purpose or end up doing the same thing and are there for historical reasons, and I tried to keep tests as identical to the way they were before as possible. IIRC the method was created where previously the code was just duplicated around, so there's some progress :), but I didn't want to change the logic of any test beyond what was strictly necessary.

##########
File path: solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
##########
@@ -790,6 +790,33 @@ public void simRemoveReplica(String nodeId, String collection, String coreNodeNa
   }
 
   /**
+<<<<<<< HEAD
+=======
+   * Save clusterstate.json to {@link DistribStateManager}.
+   * @return saved state
+   */
+  private ClusterState saveClusterState(ClusterState state) throws IOException {
+    ensureNotClosed();
+
+    // TODO: this method is emptied of its content in order to compile. We're not saving the cluster state that has to be saved collection per collection in separate state.json files.
+    // TODO: DO NOT CHECK THIS IN. Check with AB how to update sim to stateFormat 2
+
+//    byte[] data = Utils.toJSON(state);
+//    try {
+//      VersionedData oldData = stateManager.getData(ZkStateReader.CLUSTER_STATE);
+//      int version = oldData != null ? oldData.getVersion() : 0;
+//      assert clusterStateVersion == version : "local clusterStateVersion out of sync";
+//      stateManager.setData(ZkStateReader.CLUSTER_STATE, data, version);
+//      log.debug("** saved cluster state version {}", version);
+//      clusterStateVersion++;
+//    } catch (Exception e) {
+//      throw new IOException(e);
+//    }
+    return state;
+  }
+
+  /**
+>>>>>>> SOLR-12823: remove /clusterstate.json

Review comment:
       Ouch.




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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429812067



##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
##########
@@ -160,9 +159,6 @@ public void call(ClusterState state, ZkNodeProps message, NamedList results) thr
       Map<String, Object> propMap = new HashMap<>();
       propMap.put(Overseer.QUEUE_OPERATION, CREATE.toString());
       propMap.put("fromApi", "true"); // mostly true.  Prevents autoCreated=true in the collection state.
-      if (properties.get(STATE_FORMAT) == null) {

Review comment:
       Good point. Will update.




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



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


[GitHub] [lucene-solr] murblanc commented on pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#issuecomment-631124718


   > Didn't get all the way through the tests, so it might be there even though I didn't see it - would be nice to have a test that makes sure an older client that still sends `stateFormat=2` on a collection creation request would succeed.
   
   No there's nothing regarding this.
   
   Tested manually creating a collection using http://localhost:8983/solr/admin/collections?action=CREATE&name=sf2&numShards=1&replicationFactor=1&maxShardsPerNode=9999&collection.configName=_default&statedkjhsdfFormat=1 and it works.
   Note the unknown parameter at the end (stateFormat is unknown as well now).
   
   Is there an existing test when I can add this? I didn't find one.
   Or did you mean testing in a different way?


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



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


[GitHub] [lucene-solr] madrob commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r427580023



##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +493,40 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (data.length < 5) {

Review comment:
       Can we be more rigorous on the test here? 

##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
##########
@@ -236,21 +235,19 @@ public void call(ClusterState clusterState, ZkNodeProps message, NamedList resul
         }
 
         String baseUrl = zkStateReader.getBaseUrlForNodeName(nodeName);
-        //in the new mode, create the replica in clusterstate prior to creating the core.
+        //create the replica in clusterstate (i.e. ZK) prior to creating the core.

Review comment:
       s/clusterstate/state?

##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +493,40 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (data.length < 5) {
+        // less than 5 chars is empty (it's likely just "{}"). This log will only occur once.
+        log.warn("{} no longer supported starting with Solr 9. Found empty file on Zookeeper, deleting it.", ZkStateReader.UNSUPPORTED_CLUSTER_STATE);
+        zkClient.delete(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, -1, true);
+      } else {
+        // /clusterstate.json not empty: refuse to start but do not automatically delete. A bit of a pain but user shouldn't
+        // have older collections at this stage anyway.
+        String message = ZkStateReader.UNSUPPORTED_CLUSTER_STATE + " no longer supported starting with Solr 9. "
+            + "It is present and not empty. Cannot start Solr. Please first migrate collections to stateFormat=2 using an "
+            + "older version of Solr or if you don't care about the data then delete the file from "
+            + "Zookeeper using a command line tool, for example: bin/solr zk rm /clusterstate.json -z host:port";
+        log.error(message);
+        throw new SolrException(SolrException.ErrorCode.INVALID_STATE, message);
+      }
+    } catch (KeeperException e) {
+      log.error("", e);

Review comment:
       What are the conditions where this happens? log-and-throw is usually an anti-pattern.

##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
##########
@@ -468,36 +467,6 @@ void checkResults(String label, NamedList<Object> results, boolean failureIsFata
     }
   }
 
-
-  //TODO should we not remove in the next release ?
-  private void migrateStateFormat(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {

Review comment:
       Yay!




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



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


[GitHub] [lucene-solr] janhoy commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
janhoy commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429775163



##########
File path: solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
##########
@@ -790,6 +790,33 @@ public void simRemoveReplica(String nodeId, String collection, String coreNodeNa
   }
 
   /**
+<<<<<<< HEAD
+=======
+   * Save clusterstate.json to {@link DistribStateManager}.
+   * @return saved state
+   */
+  private ClusterState saveClusterState(ClusterState state) throws IOException {
+    ensureNotClosed();
+
+    // TODO: this method is emptied of its content in order to compile. We're not saving the cluster state that has to be saved collection per collection in separate state.json files.
+    // TODO: DO NOT CHECK THIS IN. Check with AB how to update sim to stateFormat 2
+
+//    byte[] data = Utils.toJSON(state);
+//    try {
+//      VersionedData oldData = stateManager.getData(ZkStateReader.CLUSTER_STATE);
+//      int version = oldData != null ? oldData.getVersion() : 0;
+//      assert clusterStateVersion == version : "local clusterStateVersion out of sync";
+//      stateManager.setData(ZkStateReader.CLUSTER_STATE, data, version);
+//      log.debug("** saved cluster state version {}", version);
+//      clusterStateVersion++;
+//    } catch (Exception e) {
+//      throw new IOException(e);
+//    }
+    return state;
+  }
+
+  /**
+>>>>>>> SOLR-12823: remove /clusterstate.json

Review comment:
       ???

##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +494,41 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (Arrays.equals("{}".getBytes(StandardCharsets.UTF_8), data)) {
+        // Empty json. This log will only occur once.
+        log.warn("{} no longer supported starting with Solr 9. Found empty file on Zookeeper, deleting it.", ZkStateReader.UNSUPPORTED_CLUSTER_STATE);
+        zkClient.delete(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, -1, true);

Review comment:
       I was thinking of the rolling upgrade scenario - if someone upgrades from 8.x to 9.0 one node at a time. Then the first node upgraded will delete /clusterstate.json. Will that cause any kind of failures or exceptions in the remaining nodes, if they have a watch on the znode or something?
   
   A way to mitigate it could be to let only the Overseer do the delete, and tell people to upgrade overseer last?

##########
File path: solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
##########
@@ -160,9 +159,6 @@ public void call(ClusterState state, ZkNodeProps message, NamedList results) thr
       Map<String, Object> propMap = new HashMap<>();
       propMap.put(Overseer.QUEUE_OPERATION, CREATE.toString());
       propMap.put("fromApi", "true"); // mostly true.  Prevents autoCreated=true in the collection state.
-      if (properties.get(STATE_FORMAT) == null) {

Review comment:
       What happens if someone backs up a 8.5 collection with stateFormat=1 and then tries to restore in 9.0? Not very likely since that collection was probably created pre-7.0 and it would not load in 9.0 anyway. But should we simply throw an exception here if STATE_FORMAT is 1?

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java
##########
@@ -138,8 +138,7 @@ private ClusterState fetchClusterState(SolrClient client, String collection, Map
     Set<String> liveNodes = new HashSet((List<String>)(cluster.get("live_nodes")));
     this.liveNodes = liveNodes;
     liveNodesTimestamp = System.nanoTime();
-    //TODO SOLR-11877 we don't know the znode path; CLUSTER_STATE is probably wrong leading to bad stateFormat

Review comment:
       Remember to close SOLR-11877 after this

##########
File path: solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
##########
@@ -210,47 +200,42 @@ public boolean liveNodesContain(String name) {
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
-    sb.append("znodeVersion: ").append(znodeVersion);
-    sb.append("\n");
     sb.append("live nodes:").append(liveNodes);
     sb.append("\n");
     sb.append("collections:").append(collectionStates);
     return sb.toString();
   }
 
-  public static ClusterState load(Integer version, byte[] bytes, Set<String> liveNodes) {
-    return load(version, bytes, liveNodes, ZkStateReader.CLUSTER_STATE);
-  }
   /**
-   * Create ClusterState from json string that is typically stored in zookeeper.
+   * Create a ClusterState from Json.
    * 
-   * @param version zk version of the clusterstate.json file (bytes)
-   * @param bytes clusterstate.json as a byte array
+   * @param bytes a byte array of a Json representation of a mapping from collection name to the Json representation of a
+   *              {@link DocCollection} as written by {@link #write(JSONWriter)}. It can represent
+   *              one or more collections.
    * @param liveNodes list of live nodes
    * @return the ClusterState
    */
-  public static ClusterState load(Integer version, byte[] bytes, Set<String> liveNodes, String znode) {
-    // System.out.println("######## ClusterState.load:" + (bytes==null ? null : new String(bytes)));
+  public static ClusterState createFromJson(int version, byte[] bytes, Set<String> liveNodes) {
     if (bytes == null || bytes.length == 0) {
-      return new ClusterState(version, liveNodes, Collections.<String, DocCollection>emptyMap());
+      return new ClusterState(liveNodes, Collections.<String, DocCollection>emptyMap());
     }
     Map<String, Object> stateMap = (Map<String, Object>) Utils.fromJSON(bytes);
-    return load(version, stateMap, liveNodes, znode);
+    return createFromData(version, stateMap, liveNodes);
   }
 
-  public static ClusterState load(Integer version, Map<String, Object> stateMap, Set<String> liveNodes, String znode) {
+  public static ClusterState createFromData(int version, Map<String, Object> stateMap, Set<String> liveNodes) {

Review comment:
       Would `createFromMap` be a more descriptive name?

##########
File path: solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java
##########
@@ -43,8 +44,12 @@
   public static void beforeClass() {
 
   }
-  
+
   @Test
+  @Ignore
+  // This test doesn't work (anymore) following https://issues.apache.org/jira/browse/SOLR-12823

Review comment:
       We should have plenty of coverage elsewhere, so +1 to remove this test?

##########
File path: solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java
##########
@@ -27,8 +27,10 @@
 import org.apache.solr.core.CoreContainer;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
+// TODO: this class tries to test Zookeeper using Solr abstractions, but ZK implies the code is running in cloud mode. It doesn't work.

Review comment:
       Perhaps MiniSolrCloud should have a `.withChroot` option which can be asserted somewhere? What about deleting this test and create a separate followup JIRA "Add zk chroot test" which can be fixed as a followup? 

##########
File path: solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
##########
@@ -181,16 +180,21 @@ public void close() {
       zkStateReader.close();
     }
 
+    /**
+     * Create a collection.
+     * Note there's a similar but slightly different {@link OverseerTest#createCollection(String, int)}.

Review comment:
       Did you consider alernatives to creating a new method with same signature and 95% same code? If duplication is necessary perhaps give the new method a more descriptive name?




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



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


[GitHub] [lucene-solr] janhoy commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
janhoy commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429829357



##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +494,41 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (Arrays.equals("{}".getBytes(StandardCharsets.UTF_8), data)) {
+        // Empty json. This log will only occur once.
+        log.warn("{} no longer supported starting with Solr 9. Found empty file on Zookeeper, deleting it.", ZkStateReader.UNSUPPORTED_CLUSTER_STATE);
+        zkClient.delete(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, -1, true);

Review comment:
       Running 8.x nodes might get a callback since they have a watch. Have not checked the callback handling code, but if the code don't handle DELETE operation then either nothing happens or an exception is thrown...




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



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


[GitHub] [lucene-solr] murblanc edited a comment on pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc edited a comment on pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#issuecomment-631124718


   > Didn't get all the way through the tests, so it might be there even though I didn't see it - would be nice to have a test that makes sure an older client that still sends `stateFormat=2` on a collection creation request would succeed.
   
   No there's nothing regarding this.
   
   Tested manually creating a collection using http://localhost:8983/solr/admin/collections?action=CREATE&name=sf2&numShards=1&replicationFactor=1&maxShardsPerNode=9999&collection.configName=_default&statedkjhsdfFormat=1 and it works.
   Note the unknown parameter at the end (stateFormat is unknown as well now).
   
   Is there an existing test where I can add this? I didn't find one.
   Or did you mean testing in a different way?


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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429811700



##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +494,41 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (Arrays.equals("{}".getBytes(StandardCharsets.UTF_8), data)) {
+        // Empty json. This log will only occur once.
+        log.warn("{} no longer supported starting with Solr 9. Found empty file on Zookeeper, deleting it.", ZkStateReader.UNSUPPORTED_CLUSTER_STATE);
+        zkClient.delete(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, -1, true);

Review comment:
       I don't think that's an issue: if /clusterstate.json is non empty, no node running 9.0 will start.
   If /clusterstate.json exists and is empty, the first starting node on 9.0 will delete it as it starts.
   8.x nodes that might start afterwards (who knows) will I believe recreate the file (that will be deleted again when a 9.0 node starts).




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



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


[GitHub] [lucene-solr] janhoy commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
janhoy commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429831171



##########
File path: solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
##########
@@ -210,47 +200,42 @@ public boolean liveNodesContain(String name) {
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
-    sb.append("znodeVersion: ").append(znodeVersion);
-    sb.append("\n");
     sb.append("live nodes:").append(liveNodes);
     sb.append("\n");
     sb.append("collections:").append(collectionStates);
     return sb.toString();
   }
 
-  public static ClusterState load(Integer version, byte[] bytes, Set<String> liveNodes) {
-    return load(version, bytes, liveNodes, ZkStateReader.CLUSTER_STATE);
-  }
   /**
-   * Create ClusterState from json string that is typically stored in zookeeper.
+   * Create a ClusterState from Json.
    * 
-   * @param version zk version of the clusterstate.json file (bytes)
-   * @param bytes clusterstate.json as a byte array
+   * @param bytes a byte array of a Json representation of a mapping from collection name to the Json representation of a
+   *              {@link DocCollection} as written by {@link #write(JSONWriter)}. It can represent
+   *              one or more collections.
    * @param liveNodes list of live nodes
    * @return the ClusterState
    */
-  public static ClusterState load(Integer version, byte[] bytes, Set<String> liveNodes, String znode) {
-    // System.out.println("######## ClusterState.load:" + (bytes==null ? null : new String(bytes)));
+  public static ClusterState createFromJson(int version, byte[] bytes, Set<String> liveNodes) {
     if (bytes == null || bytes.length == 0) {
-      return new ClusterState(version, liveNodes, Collections.<String, DocCollection>emptyMap());
+      return new ClusterState(liveNodes, Collections.<String, DocCollection>emptyMap());
     }
     Map<String, Object> stateMap = (Map<String, Object>) Utils.fromJSON(bytes);
-    return load(version, stateMap, liveNodes, znode);
+    return createFromData(version, stateMap, liveNodes);
   }
 
-  public static ClusterState load(Integer version, Map<String, Object> stateMap, Set<String> liveNodes, String znode) {
+  public static ClusterState createFromData(int version, Map<String, Object> stateMap, Set<String> liveNodes) {

Review comment:
       I don't have strong feeelings, just that 'data' could be anything :) 




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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r430012046



##########
File path: solr/core/src/java/org/apache/solr/cloud/ZkController.java
##########
@@ -491,6 +494,41 @@ public boolean isClosed() {
     assert ObjectReleaseTracker.track(this);
   }
 
+  /**
+   * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
+   * a helpful message regarding collection migration.</p>
+   *
+   * <p>If /clusterstate.json exists and is empty, it is removed.</p>
+   */
+  private void checkNoOldClusterstate(final SolrZkClient zkClient) throws InterruptedException {
+    try {
+      if (!zkClient.exists(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, true)) {
+        return;
+      }
+
+      final byte[] data = zkClient.getData(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, null, null, true);
+
+      if (Arrays.equals("{}".getBytes(StandardCharsets.UTF_8), data)) {
+        // Empty json. This log will only occur once.
+        log.warn("{} no longer supported starting with Solr 9. Found empty file on Zookeeper, deleting it.", ZkStateReader.UNSUPPORTED_CLUSTER_STATE);
+        zkClient.delete(ZkStateReader.UNSUPPORTED_CLUSTER_STATE, -1, true);

Review comment:
       Looking at pre PR master branch.
   
   The watcher on /clusterstate.json is an instance of LegacyClusterStateWatcher (subclass of ZkStateReader).
   The watcher processing is done in refreshAndWatch() that calls ZkStateReader.refreshLegacyClusterState() and does some exception handling.
   
   Even though refreshAndWatch() handles KeeperException.NoNodeException by throwing a SolrException SERVICE_UNAVAILABLE, this never happens: refreshLegacyClusterState() catches that exception, a comment says "Ignore missing legacy clusterstate.json." and the catch builds what would be an empty clusterstate.
   
   We should be fine.

##########
File path: solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
##########
@@ -210,47 +200,42 @@ public boolean liveNodesContain(String name) {
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
-    sb.append("znodeVersion: ").append(znodeVersion);
-    sb.append("\n");
     sb.append("live nodes:").append(liveNodes);
     sb.append("\n");
     sb.append("collections:").append(collectionStates);
     return sb.toString();
   }
 
-  public static ClusterState load(Integer version, byte[] bytes, Set<String> liveNodes) {
-    return load(version, bytes, liveNodes, ZkStateReader.CLUSTER_STATE);
-  }
   /**
-   * Create ClusterState from json string that is typically stored in zookeeper.
+   * Create a ClusterState from Json.
    * 
-   * @param version zk version of the clusterstate.json file (bytes)
-   * @param bytes clusterstate.json as a byte array
+   * @param bytes a byte array of a Json representation of a mapping from collection name to the Json representation of a
+   *              {@link DocCollection} as written by {@link #write(JSONWriter)}. It can represent
+   *              one or more collections.
    * @param liveNodes list of live nodes
    * @return the ClusterState
    */
-  public static ClusterState load(Integer version, byte[] bytes, Set<String> liveNodes, String znode) {
-    // System.out.println("######## ClusterState.load:" + (bytes==null ? null : new String(bytes)));
+  public static ClusterState createFromJson(int version, byte[] bytes, Set<String> liveNodes) {
     if (bytes == null || bytes.length == 0) {
-      return new ClusterState(version, liveNodes, Collections.<String, DocCollection>emptyMap());
+      return new ClusterState(liveNodes, Collections.<String, DocCollection>emptyMap());
     }
     Map<String, Object> stateMap = (Map<String, Object>) Utils.fromJSON(bytes);
-    return load(version, stateMap, liveNodes, znode);
+    return createFromData(version, stateMap, liveNodes);
   }
 
-  public static ClusterState load(Integer version, Map<String, Object> stateMap, Set<String> liveNodes, String znode) {
+  public static ClusterState createFromData(int version, Map<String, Object> stateMap, Set<String> liveNodes) {

Review comment:
       `createFromCollectionMap`




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



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


[GitHub] [lucene-solr] murblanc commented on pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#issuecomment-641124398


   @madrob @janhoy @noblepaul would one of you be able to merge this eventually?
   I'm afraid this is going to sit here forever otherwise...


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



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


[GitHub] [lucene-solr] murblanc commented on a change in pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on a change in pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#discussion_r429816694



##########
File path: solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java
##########
@@ -27,8 +27,10 @@
 import org.apache.solr.core.CoreContainer;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
+// TODO: this class tries to test Zookeeper using Solr abstractions, but ZK implies the code is running in cloud mode. It doesn't work.

Review comment:
       I agree (on the followup option).
   I tried to see if there's a way to do the chroot on the mini cluster or elsewhere but nothing obvious came after an hour or two of hacking, that's why I suggest to leave it for later.




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



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


[GitHub] [lucene-solr] murblanc commented on pull request #1528: SOLR-12823: remove /clusterstate.json

Posted by GitBox <gi...@apache.org>.
murblanc commented on pull request #1528:
URL: https://github.com/apache/lucene-solr/pull/1528#issuecomment-631131151


   Rebased on master. The recent fix to SOLR-14486 solved the failures in the autoscaling sim tests.
   
   I'm submitting this officially for review now (moving it out of draft PR).
   
   Note there are two tests that are failing (that I marked @Ignore) and that IMO should either be removed or deserve their own separate Jira for fixing (that I will create once this is merged). I suspect fixes are non trivial:
   - TestZkChroot: I didn't find ways to reasonably start a test server with a different chroot than /solr to test if it's writing data to the right place in ZooKeeper. Played a while with MiniSolrCloudCluster but chroot seems to be quite hard coded deep in the test framework (passing an already chrooted ZK server defeats the purpose).
   - BasicZkTest (I didn't dig further).


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



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