You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2018/10/03 13:38:15 UTC

lucene-solr:master: SOLR-12822: /autoscaling/suggestions to include suggestion to add-replica for lost replicas

Repository: lucene-solr
Updated Branches:
  refs/heads/master abace2987 -> d8e40796e


SOLR-12822: /autoscaling/suggestions to include suggestion to add-replica for lost replicas


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d8e40796
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d8e40796
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d8e40796

Branch: refs/heads/master
Commit: d8e40796e224987f1db696a57569b712aaf1e503
Parents: abace29
Author: Noble Paul <no...@apache.org>
Authored: Wed Oct 3 23:37:58 2018 +1000
Committer: Noble Paul <no...@apache.org>
Committed: Wed Oct 3 23:37:58 2018 +1000

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../solrj/cloud/autoscaling/PolicyHelper.java   |  53 ++++++--
 .../solrj/cloud/autoscaling/ReplicaCount.java   |  13 +-
 .../solrj/cloud/autoscaling/Suggester.java      |   6 +-
 .../solrj/cloud/autoscaling/Suggestion.java     |   7 +-
 .../apache/solr/common/cloud/DocCollection.java |   9 ++
 .../java/org/apache/solr/common/util/Utils.java |  21 +++-
 .../solr/autoscaling/testAddMissingReplica.json | 123 +++++++++++++++++++
 .../solrj/cloud/autoscaling/TestPolicy.java     |   5 +
 .../solrj/cloud/autoscaling/TestPolicy2.java    |  34 +++++
 10 files changed, 259 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index af95736..36a63d2 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -115,6 +115,8 @@ New Features
 
 * SOLR-11522: /autoscaling/suggestions now include rebalance options as well even if there are no violations (noble)
 
+* SOLR-12822: /autoscaling/suggestions to include suggestion to add-replica for lost replicas (noble)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
index 123d144..0f4af70 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
@@ -226,7 +226,7 @@ public class PolicyHelper {
 
   public static List<Suggester.SuggestionInfo> getSuggestions(AutoScalingConfig autoScalingConf,
                                                               SolrCloudManager cloudManager) {
-    return getSuggestions(autoScalingConf, cloudManager, 20);
+    return getSuggestions(autoScalingConf, cloudManager, 50);
   }
 
   public static List<Suggester.SuggestionInfo> getSuggestions(AutoScalingConfig autoScalingConf,
@@ -244,13 +244,52 @@ public class PolicyHelper {
       tagType.getSuggestions(ctx.setViolation(violation));
       ctx.violation = null;
     }
-    if (ctx.getSuggestions().size() < max) {
-      suggestOptimizations(ctx);
+
+    if (ctx.needMore()) {
+      try {
+        addMissingReplicas(cloudManager, ctx);
+      } catch (IOException e) {
+        log.error("Unable to fetch cluster state", e);
+      }
+    }
+
+    if (ctx.needMore()) {
+      suggestOptimizations(ctx, Math.min(ctx.max - ctx.getSuggestions().size(), 10));
     }
     return ctx.getSuggestions();
   }
 
-  private static void suggestOptimizations(Suggestion.Ctx ctx) {
+  private static void addMissingReplicas(SolrCloudManager cloudManager, Suggestion.Ctx ctx) throws IOException {
+    cloudManager.getClusterStateProvider().getClusterState().forEachCollection(coll -> coll.forEach(slice -> {
+          ReplicaCount replicaCount = new ReplicaCount();
+          slice.forEach(replica -> {
+            if (replica.getState() == Replica.State.ACTIVE || replica.getState() == Replica.State.RECOVERING) {
+              replicaCount.increment(replica.getType());
+            }
+          });
+          addMissingReplicas(replicaCount, coll, slice.getName(), Replica.Type.NRT, ctx);
+          addMissingReplicas(replicaCount, coll, slice.getName(), Replica.Type.PULL, ctx);
+          addMissingReplicas(replicaCount, coll, slice.getName(), Replica.Type.TLOG, ctx);
+        }
+    ));
+  }
+
+  private static void addMissingReplicas(ReplicaCount count, DocCollection coll, String shard, Replica.Type type, Suggestion.Ctx ctx) {
+    int delta = count.delta(coll.getExpectedReplicaCount(type, 0), type);
+    for (; ; ) {
+      if (delta >= 0) break;
+      SolrRequest suggestion = ctx.addSuggestion(
+          ctx.session.getSuggester(ADDREPLICA)
+              .hint(Hint.REPLICATYPE, type)
+              .hint(Hint.COLL_SHARD, new Pair(coll.getName(), shard)), "repair");
+      if (suggestion == null) return;
+      delta++;
+    }
+  }
+
+
+  private static void suggestOptimizations(Suggestion.Ctx ctx, int count) {
+    int maxTotalSuggestions = ctx.getSuggestions().size() + count;
     List<Row> matrix = ctx.session.matrix;
     if (matrix.isEmpty()) return;
     for (int i = 0; i < matrix.size(); i++) {
@@ -261,13 +300,13 @@ public class PolicyHelper {
         e.setValue(FreeDiskVariable.getSortedShards(Collections.singletonList(row), e.getValue(), e.getKey()));
       }
       for (Map.Entry<String, Collection<String>> e : collVsShards.entrySet()) {
-        if (!ctx.needMore()) break;
+        if (ctx.getSuggestions().size() >= maxTotalSuggestions) break;
         for (String shard : e.getValue()) {
-          if (!ctx.needMore()) break;
           Suggester suggester = ctx.session.getSuggester(MOVEREPLICA)
               .hint(Hint.COLL_SHARD, new Pair<>(e.getKey(), shard))
               .hint(Hint.SRC_NODE, row.node);
-          ctx.addSuggestion(suggester);
+          ctx.addSuggestion(suggester, "improvement");
+          if (ctx.getSuggestions().size() >= maxTotalSuggestions) break;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java
index 87fcf5a..e168ff9 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java
@@ -70,7 +70,11 @@ class ReplicaCount  implements MapWriter {
   }
 
   void increment(ReplicaInfo info) {
-    switch (info.getType()) {
+    increment(info.getType());
+  }
+
+  public void increment(Replica.Type type) {
+    switch (type) {
       case NRT:
         nrt++;
         break;
@@ -97,4 +101,11 @@ class ReplicaCount  implements MapWriter {
   public void reset() {
     nrt = tlog = pull = 0;
   }
+
+  public int delta(int expectedReplicaCount, Replica.Type type) {
+    if (type == Replica.Type.NRT) return (int) (nrt - expectedReplicaCount);
+    if (type == Replica.Type.PULL) return (int) (pull - expectedReplicaCount);
+    if (type == Replica.Type.TLOG) return (int) (tlog - expectedReplicaCount);
+    throw new RuntimeException("NO type");
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java
index 9f42b9f..0071978 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java
@@ -189,12 +189,14 @@ public abstract class Suggester implements MapWriter {
   }
 
   public static class SuggestionInfo implements MapWriter {
+    String type;
     Violation violation;
     SolrRequest operation;
 
-    public SuggestionInfo(Violation violation, SolrRequest op) {
+    public SuggestionInfo(Violation violation, SolrRequest op, String type) {
       this.violation = violation;
       this.operation = op;
+      this.type = type;
     }
 
     public SolrRequest getOperation() {
@@ -207,7 +209,7 @@ public abstract class Suggester implements MapWriter {
 
     @Override
     public void writeMap(EntryWriter ew) throws IOException {
-      ew.put("type", violation == null ? "improvement" : "violation");
+      ew.put("type", type);
       if(violation!= null) ew.put("violation",
           new ConditionalMapWriter(violation,
               (k, v) -> !"violatingReplicas".equals(k)));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
index 8f120e2..45d4582 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggestion.java
@@ -36,13 +36,16 @@ public class Suggestion {
     public Policy.Session session;
     public Violation violation;
     private List<Suggester.SuggestionInfo> suggestions = new ArrayList<>();
-
     SolrRequest addSuggestion(Suggester suggester) {
+      return addSuggestion(suggester, "violation");
+    }
+
+    SolrRequest addSuggestion(Suggester suggester, String type) {
       SolrRequest op = suggester.getSuggestion();
       if (op != null) {
         session = suggester.getSession();
         suggestions.add(new Suggester.SuggestionInfo(violation,
-            ((V2RequestSupport) op.setUseV2(true)).getV2Request()));
+            ((V2RequestSupport) op.setUseV2(true)).getV2Request(), type));
       }
       return op;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index ab250a6..adf0211 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -406,4 +406,13 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
   public String getPolicyName() {
     return policy;
   }
+
+  public int getExpectedReplicaCount(Replica.Type type, int def) {
+    Integer result = null;
+    if (type == Replica.Type.NRT) result = numNrtReplicas;
+    if (type == Replica.Type.PULL) result = numPullReplicas;
+    if (type == Replica.Type.TLOG) result = numTlogReplicas;
+    return result == null ? def : result;
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index c8274f3..ac3f120 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -45,6 +45,7 @@ import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.function.BiConsumer;
+import java.util.function.Function;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -265,7 +266,23 @@ public class Utils {
 
   public static Object fromJSON(InputStream is){
     try {
-      return new ObjectBuilder(getJSONParser((new InputStreamReader(is, StandardCharsets.UTF_8)))).getVal();
+      return STANDARDOBJBUILDER.apply(getJSONParser((new InputStreamReader(is, StandardCharsets.UTF_8)))).getVal();
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error", e);
+    }
+  }
+
+  public static final Function<JSONParser, ObjectBuilder> STANDARDOBJBUILDER = jsonParser -> {
+    try {
+      return new ObjectBuilder(jsonParser);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  };
+
+  public static Object fromJSON(InputStream is, Function<JSONParser, ObjectBuilder> objBuilderProvider) {
+    try {
+      return objBuilderProvider.apply(getJSONParser((new InputStreamReader(is, StandardCharsets.UTF_8)))).getVal();
     } catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error", e);
     }
@@ -293,7 +310,7 @@ public class Utils {
 
   public static Object fromJSONString(String json)  {
     try {
-      return new ObjectBuilder(getJSONParser(new StringReader(json))).getVal();
+      return STANDARDOBJBUILDER.apply(getJSONParser(new StringReader(json))).getVal();
     } catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error", e);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/test-files/solrj/solr/autoscaling/testAddMissingReplica.json
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test-files/solrj/solr/autoscaling/testAddMissingReplica.json b/solr/solrj/src/test-files/solrj/solr/autoscaling/testAddMissingReplica.json
new file mode 100644
index 0000000..6046945
--- /dev/null
+++ b/solr/solrj/src/test-files/solrj/solr/autoscaling/testAddMissingReplica.json
@@ -0,0 +1,123 @@
+{
+  "responseHeader":{
+    "status":0,
+    "QTime":23},
+  "diagnostics":{
+    "sortedNodes":[{
+      "node":"10.0.0.80:7575_solr",
+      "isLive":true,
+      "cores":1.0,
+      "freedisk":673.2483978271484,
+      "totaldisk":1037.938980102539,
+      "replicas":{"gettingstarted":{"shard1":[{
+        "core_node4":{
+          "core":"gettingstarted_shard1_replica_n1",
+          "shard":"shard1",
+          "collection":"gettingstarted",
+          "node_name":"10.0.0.80:7575_solr",
+          "type":"NRT",
+          "leader":"true",
+          "base_url":"http://10.0.0.80:7575/solr",
+          "state":"active",
+          "force_set_state":"false",
+          "INDEX.sizeInGB":6.426125764846802E-8}}]}}}
+    ,{
+        "node":"10.0.0.80:8983_solr",
+        "isLive":true,
+        "cores":1.0,
+        "freedisk":673.2483940124512,
+        "totaldisk":1037.938980102539,
+        "replicas":{"gettingstarted":{"shard2":[{
+          "core_node8":{
+            "core":"gettingstarted_shard2_replica_n5",
+            "shard":"shard2",
+            "collection":"gettingstarted",
+            "node_name":"10.0.0.80:8983_solr",
+            "type":"NRT",
+            "leader":"true",
+            "base_url":"http://10.0.0.80:8983/solr",
+            "state":"active",
+            "force_set_state":"false",
+            "INDEX.sizeInGB":6.426125764846802E-8}}]}}}
+    ,{
+        "node":"10.0.0.80:8984_solr",
+        "isLive":true,
+        "cores":1.0,
+        "freedisk":673.2483901977539,
+        "totaldisk":1037.938980102539,
+        "replicas":{"gettingstarted":{"shard1":[{
+          "core_node6":{
+            "core":"gettingstarted_shard1_replica_n2",
+            "shard":"shard1",
+            "collection":"gettingstarted",
+            "node_name":"10.0.0.80:8984_solr",
+            "type":"NRT",
+            "base_url":"http://10.0.0.80:8984/solr",
+            "state":"active",
+            "force_set_state":"false",
+            "INDEX.sizeInGB":6.426125764846802E-8}}]}}}],
+    "liveNodes":["10.0.0.80:7575_solr",
+      "10.0.0.80:8983_solr",
+      "10.0.0.80:8984_solr"],
+    "violations":[],
+    "config":{
+      "cluster-preferences":[{
+        "minimize":"cores",
+        "precision":1}
+      ,{
+          "maximize":"freedisk"}]}},
+
+  "cluster":{
+    "collections":{
+      "gettingstarted":{
+        "pullReplicas":"0",
+        "replicationFactor":"2",
+        "shards":{
+          "shard1":{
+            "range":"80000000-ffffffff",
+            "state":"active",
+            "replicas":{
+              "core_node4":{
+                "core":"gettingstarted_shard1_replica_n1",
+                "base_url":"http://10.0.0.80:7575/solr",
+                "node_name":"10.0.0.80:7575_solr",
+                "state":"active",
+                "type":"NRT",
+                "force_set_state":"false",
+                "leader":"true"},
+              "core_node6":{
+                "core":"gettingstarted_shard1_replica_n2",
+                "base_url":"http://10.0.0.80:8984/solr",
+                "node_name":"10.0.0.80:8984_solr",
+                "state":"active",
+                "type":"NRT",
+                "force_set_state":"false"}}},
+          "shard2":{
+            "range":"0-7fffffff",
+            "state":"active",
+            "replicas":{
+              "core_node7":{
+                "core":"gettingstarted_shard2_replica_n3",
+                "base_url":"http://10.0.0.80:7574/solr",
+                "node_name":"10.0.0.80:7574_solr",
+                "state":"down",
+                "type":"NRT",
+                "force_set_state":"false"},
+              "core_node8":{
+                "core":"gettingstarted_shard2_replica_n5",
+                "base_url":"http://10.0.0.80:8983/solr",
+                "node_name":"10.0.0.80:8983_solr",
+                "state":"active",
+                "type":"NRT",
+                "force_set_state":"false",
+                "leader":"true"}}}},
+        "router":{"name":"compositeId"},
+        "maxShardsPerNode":"-1",
+        "autoAddReplicas":"false",
+        "nrtReplicas":"2",
+        "tlogReplicas":"0",
+        "znodeVersion":12,
+        "configName":"gettingstarted"}},
+    "live_nodes":["10.0.0.80:8983_solr",
+      "10.0.0.80:7575_solr",
+      "10.0.0.80:8984_solr"]}}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
index 23184a0..ec3c56c 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
@@ -1345,6 +1345,11 @@ public class TestPolicy extends SolrTestCaseJ4 {
       public ClusterStateProvider getClusterStateProvider() {
         return new DelegatingClusterStateProvider(null) {
           @Override
+          public ClusterState getClusterState() throws IOException {
+            return ClusterState.load(0,new HashMap<>(), getLiveNodes(),"/clusterstate.json");
+          }
+
+          @Override
           public Set<String> getLiveNodes() {
             return new HashSet<>((Collection<String>) m.get("liveNodes"));
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8e40796/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
index 07ce391..391b210 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -338,6 +339,7 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
       }
     };
     return new DelegatingCloudManager(null) {
+      ClusterState clusterState = null;
       @Override
       public NodeStateProvider getNodeStateProvider() {
         return nodeStateProvider;
@@ -345,7 +347,24 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
 
       @Override
       public ClusterStateProvider getClusterStateProvider() {
+        if (clusterState == null) {
+          Map map = (Map) Utils.getObjectByPath (m, false, "cluster/collections");
+          if (map == null) map = new HashMap<>();
+          clusterState = ClusterState.load(0, map, liveNodes, "/clusterstate.json");
+        }
+
         return new DelegatingClusterStateProvider(null) {
+
+          @Override
+          public ClusterState getClusterState() throws IOException {
+            return clusterState;
+          }
+
+          @Override
+          public ClusterState.CollectionRef getState(String collection) {
+            return clusterState.getCollectionRef(collection);
+          }
+
           @Override
           public Set<String> getLiveNodes() {
             return liveNodes;
@@ -414,6 +433,21 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
 
   }
 
+  public void testAddMissingReplica() throws IOException {
+    Map<String, Object> m = (Map<String, Object>) loadFromResource("testAddMissingReplica.json");
+    SolrCloudManager cloudManagerFromDiagnostics = createCloudManagerFromDiagnostics(m);
+    AutoScalingConfig autoScalingConfig = new AutoScalingConfig((Map<String, Object>) Utils.getObjectByPath(m, false, "diagnostics/config"));
+
+    List<Suggester.SuggestionInfo> suggestions = PolicyHelper.getSuggestions(autoScalingConfig, cloudManagerFromDiagnostics);
+
+    assertEquals(1, suggestions.size());
+    assertEquals("repair", suggestions.get(0)._get("type",null));
+    assertEquals("add-replica", suggestions.get(0)._get("operation/command[0]/key",null));
+    assertEquals("shard2", suggestions.get(0)._get("operation/command/add-replica/shard",null));
+    assertEquals("NRT", suggestions.get(0)._get("operation/command/add-replica/type",null));
+
+  }
+
   public static Object loadFromResource(String file) throws IOException {
     try (InputStream is = TestPolicy2.class.getResourceAsStream("/solrj/solr/autoscaling/" + file)) {
       return Utils.fromJSON(is);