You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by "vinayakphegde (via GitHub)" <gi...@apache.org> on 2023/03/23 06:36:51 UTC

[GitHub] [solr] vinayakphegde opened a new pull request, #1485: SOLR-16507: Remove NodeStateProvider & Snitch

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

   use metric API instead of NodeStateProvider in SplitShardCmd.
   


-- 
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] vinayakphegde commented on a diff in pull request #1485: SOLR-16507: Remove NodeStateProvider & Snitch

Posted by "vinayakphegde (via GitHub)" <gi...@apache.org>.
vinayakphegde commented on code in PR #1485:
URL: https://github.com/apache/solr/pull/1485#discussion_r1146346389


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,85 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
+
+    ModifiableSolrParams params;
+    String metricName;
+    GenericSolrRequest req;
+    SolrResponse rsp;

Review Comment:
   Haha. I am working on the Linux kernel too, so mixed up my coding styles. Will change 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] dsmiley merged pull request #1485: SOLR-16507: Remove NodeStateProvider & Snitch

Posted by "dsmiley (via GitHub)" <gi...@apache.org>.
dsmiley merged PR #1485:
URL: https://github.com/apache/solr/pull/1485


-- 
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] noblepaul commented on pull request #1485: SOLR-16507: Remove NodeStateProvider & Snitch

Posted by "noblepaul (via GitHub)" <gi...@apache.org>.
noblepaul commented on PR #1485:
URL: https://github.com/apache/solr/pull/1485#issuecomment-1490213738

   > hen please share in the JIRA issue as there's a conversation there already.
   
   I shall submit a PR for the JIRA


-- 
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] dsmiley commented on a diff in pull request #1485: SOLR-16507: Remove NodeStateProvider & Snitch

Posted by "dsmiley (via GitHub)" <gi...@apache.org>.
dsmiley commented on code in PR #1485:
URL: https://github.com/apache/solr/pull/1485#discussion_r1146200128


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,85 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
+
+    ModifiableSolrParams params;
+    String metricName;
+    GenericSolrRequest req;
+    SolrResponse rsp;
+
+    metricName =
+        new StringBuilder("solr.core.")

Review Comment:
   Needless use of StringBuilder; just use simple string concatenation.



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,85 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
+
+    ModifiableSolrParams params;
+    String metricName;
+    GenericSolrRequest req;
+    SolrResponse rsp;

Review Comment:
   This is a coding style that I have only ever seen decades ago in languages like "C".  Please don't declare a bunch of variables up front; declare them at the latest possible time, and with its initial value if possible.



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,85 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
+
+    ModifiableSolrParams params;
+    String metricName;
+    GenericSolrRequest req;
+    SolrResponse rsp;
+
+    metricName =
+        new StringBuilder("solr.core.")
+            .append(collection)
+            .append(".")
+            .append(shard)
+            .append(".")
+            .append(Utils.parseMetricsReplicaName(collection, parentShardLeader.getCoreName()))
+            .append(":INDEX.sizeInBytes")
+            .toString();
+
+    params = new ModifiableSolrParams();
+    params.add("key", metricName);
+
+    req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params);
+    try {
+      rsp = cloudManager.request(req);
+    } catch (Exception e) {
+      log.error("Error occurred while checking the disk space of the node");
+      return;
+    }
+
+    if (rsp.getResponse() == null) {
       log.warn("cannot verify information for parent shard leader");
       return;
     }
-    // find the leader
-    List<Replica> lst = infos.get(collection).get(shard);
-    Double indexSize = null;
-    for (Replica info : lst) {
-      if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
-        Number size = (Number) info.get(CORE_IDX.metricsAttribute);
-        if (size == null) {
-          log.warn("cannot verify information for parent shard leader");
-          return;
-        }
-        indexSize = (Double) CORE_IDX.convertVal(size);
-        break;
-      }
+
+    NamedList<Object> response = rsp.getResponse();
+    Object value = response.findRecursive("metrics", metricName);
+    if (value == null) {
+      log.warn("cannot verify information for parent shard leader");
+      return;
+    }
+
+    Number size = (Number) value;
+    if (size == null) {
+      log.warn("cannot verify information for parent shard leader");
+      return;
     }
-    if (indexSize == null) {
-      log.warn("missing replica information for parent shard leader");
+    double indexSize = size.doubleValue();
+
+    metricName = "solr.node:CONTAINER.fs.usableSpace";
+    params = new ModifiableSolrParams();
+    params.add("key", metricName);

Review Comment:
   BTW can do all in one line:
   `SolrParams = new ModifiableSolrParams().add("key", metricName);`
   Please avoid code verbosity.



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,85 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
+
+    ModifiableSolrParams params;
+    String metricName;
+    GenericSolrRequest req;
+    SolrResponse rsp;
+
+    metricName =
+        new StringBuilder("solr.core.")
+            .append(collection)
+            .append(".")
+            .append(shard)
+            .append(".")
+            .append(Utils.parseMetricsReplicaName(collection, parentShardLeader.getCoreName()))
+            .append(":INDEX.sizeInBytes")
+            .toString();
+
+    params = new ModifiableSolrParams();
+    params.add("key", metricName);
+
+    req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params);
+    try {
+      rsp = cloudManager.request(req);
+    } catch (Exception e) {
+      log.error("Error occurred while checking the disk space of the node");
+      return;
+    }
+
+    if (rsp.getResponse() == null) {
       log.warn("cannot verify information for parent shard leader");
       return;
     }
-    // find the leader
-    List<Replica> lst = infos.get(collection).get(shard);
-    Double indexSize = null;
-    for (Replica info : lst) {
-      if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
-        Number size = (Number) info.get(CORE_IDX.metricsAttribute);
-        if (size == null) {
-          log.warn("cannot verify information for parent shard leader");
-          return;
-        }
-        indexSize = (Double) CORE_IDX.convertVal(size);
-        break;
-      }
+
+    NamedList<Object> response = rsp.getResponse();
+    Object value = response.findRecursive("metrics", metricName);
+    if (value == null) {
+      log.warn("cannot verify information for parent shard leader");
+      return;
+    }
+

Review Comment:
   Not needed; don't declare "value"



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,85 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
+
+    ModifiableSolrParams params;
+    String metricName;
+    GenericSolrRequest req;
+    SolrResponse rsp;
+
+    metricName =
+        new StringBuilder("solr.core.")
+            .append(collection)
+            .append(".")
+            .append(shard)
+            .append(".")
+            .append(Utils.parseMetricsReplicaName(collection, parentShardLeader.getCoreName()))
+            .append(":INDEX.sizeInBytes")
+            .toString();
+
+    params = new ModifiableSolrParams();
+    params.add("key", metricName);
+
+    req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params);
+    try {
+      rsp = cloudManager.request(req);
+    } catch (Exception e) {
+      log.error("Error occurred while checking the disk space of the node");
+      return;
+    }
+
+    if (rsp.getResponse() == null) {
       log.warn("cannot verify information for parent shard leader");
       return;
     }
-    // find the leader
-    List<Replica> lst = infos.get(collection).get(shard);
-    Double indexSize = null;
-    for (Replica info : lst) {
-      if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
-        Number size = (Number) info.get(CORE_IDX.metricsAttribute);
-        if (size == null) {
-          log.warn("cannot verify information for parent shard leader");
-          return;
-        }
-        indexSize = (Double) CORE_IDX.convertVal(size);
-        break;
-      }
+
+    NamedList<Object> response = rsp.getResponse();
+    Object value = response.findRecursive("metrics", metricName);
+    if (value == null) {
+      log.warn("cannot verify information for parent shard leader");
+      return;
+    }
+
+    Number size = (Number) value;
+    if (size == null) {
+      log.warn("cannot verify information for parent shard leader");
+      return;
     }
-    if (indexSize == null) {
-      log.warn("missing replica information for parent shard leader");
+    double indexSize = size.doubleValue();
+
+    metricName = "solr.node:CONTAINER.fs.usableSpace";
+    params = new ModifiableSolrParams();
+    params.add("key", metricName);
+
+    req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params);

Review Comment:
   You are doing a second request but the metrics API is rich enough to support both needs in one request.  "key" can be provided multiple times for each value needed.



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,85 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
+
+    ModifiableSolrParams params;
+    String metricName;
+    GenericSolrRequest req;
+    SolrResponse rsp;
+
+    metricName =
+        new StringBuilder("solr.core.")
+            .append(collection)
+            .append(".")
+            .append(shard)
+            .append(".")
+            .append(Utils.parseMetricsReplicaName(collection, parentShardLeader.getCoreName()))
+            .append(":INDEX.sizeInBytes")
+            .toString();
+
+    params = new ModifiableSolrParams();
+    params.add("key", metricName);
+
+    req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params);

Review Comment:
   could inline into below



-- 
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] dsmiley commented on a diff in pull request #1485: SOLR-16507: Remove NodeStateProvider & Snitch

Posted by "dsmiley (via GitHub)" <gi...@apache.org>.
dsmiley commented on code in PR #1485:
URL: https://github.com/apache/solr/pull/1485#discussion_r1146295091


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,85 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
+
+    ModifiableSolrParams params;
+    String metricName;
+    GenericSolrRequest req;
+    SolrResponse rsp;
+
+    metricName =
+        new StringBuilder("solr.core.")
+            .append(collection)
+            .append(".")
+            .append(shard)
+            .append(".")
+            .append(Utils.parseMetricsReplicaName(collection, parentShardLeader.getCoreName()))
+            .append(":INDEX.sizeInBytes")
+            .toString();
+
+    params = new ModifiableSolrParams();
+    params.add("key", metricName);
+
+    req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params);
+    try {
+      rsp = cloudManager.request(req);
+    } catch (Exception e) {
+      log.error("Error occurred while checking the disk space of the node");
+      return;
+    }
+
+    if (rsp.getResponse() == null) {
       log.warn("cannot verify information for parent shard leader");
       return;
     }
-    // find the leader
-    List<Replica> lst = infos.get(collection).get(shard);
-    Double indexSize = null;
-    for (Replica info : lst) {
-      if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
-        Number size = (Number) info.get(CORE_IDX.metricsAttribute);
-        if (size == null) {
-          log.warn("cannot verify information for parent shard leader");
-          return;
-        }
-        indexSize = (Double) CORE_IDX.convertVal(size);
-        break;
-      }
+
+    NamedList<Object> response = rsp.getResponse();
+    Object value = response.findRecursive("metrics", metricName);
+    if (value == null) {
+      log.warn("cannot verify information for parent shard leader");
+      return;
+    }
+
+    Number size = (Number) value;
+    if (size == null) {
+      log.warn("cannot verify information for parent shard leader");
+      return;
     }
-    if (indexSize == null) {
-      log.warn("missing replica information for parent shard leader");
+    double indexSize = size.doubleValue();
+
+    metricName = "solr.node:CONTAINER.fs.usableSpace";
+    params = new ModifiableSolrParams();
+    params.add("key", metricName);
+
+    req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params);

Review Comment:
   https://solr.apache.org/guide/solr/latest/deployment-guide/metrics-reporting.html#metrics-api



-- 
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] noblepaul commented on pull request #1485: SOLR-16507: Remove NodeStateProvider & Snitch

Posted by "noblepaul (via GitHub)" <gi...@apache.org>.
noblepaul commented on PR #1485:
URL: https://github.com/apache/solr/pull/1485#issuecomment-1487748631

   LGTM , `snitch` must be removed 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] dsmiley commented on a diff in pull request #1485: SOLR-16507: Remove NodeStateProvider & Snitch

Posted by "dsmiley (via GitHub)" <gi...@apache.org>.
dsmiley commented on code in PR #1485:
URL: https://github.com/apache/solr/pull/1485#discussion_r1146665022


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,52 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
-      log.warn("cannot verify information for parent shard leader");
+
+    String indexSizeMetricName =
+        "solr.core."
+            + collection
+            + "."
+            + shard
+            + "."
+            + Utils.parseMetricsReplicaName(collection, parentShardLeader.getCoreName())
+            + ":INDEX.sizeInBytes";
+    String freeDiskSpaceMetricName = "solr.node:CONTAINER.fs.usableSpace";
+
+    ModifiableSolrParams params =
+        new ModifiableSolrParams()
+            .add("key", indexSizeMetricName)
+            .add("key", freeDiskSpaceMetricName);
+    SolrResponse rsp;
+    try {
+      rsp =
+          cloudManager.request(
+              new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params));
+    } catch (Exception e) {
+      log.error("Error occurred while checking the disk space of the node");
       return;
     }
-    // find the leader
-    List<Replica> lst = infos.get(collection).get(shard);
-    Double indexSize = null;
-    for (Replica info : lst) {
-      if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
-        Number size = (Number) info.get(CORE_IDX.metricsAttribute);
-        if (size == null) {
-          log.warn("cannot verify information for parent shard leader");
-          return;
-        }
-        indexSize = (Double) CORE_IDX.convertVal(size);
-        break;
-      }
+    if (rsp.getResponse() == null) {
+      log.warn("cannot verify information for parent shard leader");
+      return;
     }

Review Comment:
   I would love to get rid of this.  There will be a response.  If it can be shown that there isn't in weird cases, I'd rather improve such weird cases so Solr code can make such guarantees without needless/verbose null checks. 
   You could not declare the "rsp" even; just immediately grab the all-important NamedList.



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,52 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
-      log.warn("cannot verify information for parent shard leader");
+
+    String indexSizeMetricName =
+        "solr.core."
+            + collection
+            + "."
+            + shard
+            + "."
+            + Utils.parseMetricsReplicaName(collection, parentShardLeader.getCoreName())

Review Comment:
   declare this as its own variable so that there isn't this silly long line breaking.  Or String.format if you prefer would still be a substantial improvement.



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java:
##########
@@ -873,42 +873,52 @@ public static void checkDiskSpace(
       SolrIndexSplitter.SplitMethod method,
       SolrCloudManager cloudManager)
       throws SolrException {
+
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
-    NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues =
-        nodeStateProvider.getNodeValues(
-            parentShardLeader.getNodeName(), Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<Replica>>> infos =
-        nodeStateProvider.getReplicaInfo(
-            parentShardLeader.getNodeName(), Collections.singletonList(CORE_IDX.metricsAttribute));
-    if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
-      log.warn("cannot verify information for parent shard leader");
+
+    String indexSizeMetricName =
+        "solr.core."
+            + collection
+            + "."
+            + shard
+            + "."
+            + Utils.parseMetricsReplicaName(collection, parentShardLeader.getCoreName())
+            + ":INDEX.sizeInBytes";
+    String freeDiskSpaceMetricName = "solr.node:CONTAINER.fs.usableSpace";
+
+    ModifiableSolrParams params =
+        new ModifiableSolrParams()
+            .add("key", indexSizeMetricName)
+            .add("key", freeDiskSpaceMetricName);
+    SolrResponse rsp;
+    try {
+      rsp =
+          cloudManager.request(
+              new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params));
+    } catch (Exception e) {
+      log.error("Error occurred while checking the disk space of the node");

Review Comment:
   always propagate the exception!



-- 
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] dsmiley commented on pull request #1485: SOLR-16507: Remove NodeStateProvider & Snitch

Posted by "dsmiley (via GitHub)" <gi...@apache.org>.
dsmiley commented on PR #1485:
URL: https://github.com/apache/solr/pull/1485#issuecomment-1482753208

   Looking good!  I think a net improvement.  After this change, NodeStateProvider and its related weird named things (Snitches) is only used in one place.
   
   WDYT @sigram ?   (BTW I tagged you for review on another unrelated PR by accident)


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