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 2019/06/03 08:41:37 UTC

[lucene-solr] branch branch_8x updated (4f79fdc -> 5c677f8)

This is an automated email from the ASF dual-hosted git repository.

noble pushed a change to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from 4f79fdc  LUCENE-8813: Ensure we never apply deletes from a closed DWPTDeleteQueue (#688)
     new dbbab57  SOLR-13504: improve autoscaling syntax by adding a nodeset attribute (#691)
     new b3caaff  SOLR-13504 improve autoscaling syntax by adding a nodeset attribute (#693)
     new 5c677f8  SOLR-13504: In autoscaling policies, use an explicit 'put : on-each'   to specify the the rules is applied on each node (#694)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 solr/CHANGES.txt                                   |   6 +
 .../cloud/autoscaling/AutoScalingHandlerTest.java  |  16 ++
 .../client/solrj/cloud/autoscaling/Clause.java     | 196 +++++++++++----
 .../client/solrj/cloud/autoscaling/Policy.java     |   2 +-
 .../client/solrj/cloud/autoscaling/Variable.java   |   5 +-
 .../solrj/cloud/autoscaling/VariableBase.java      |  11 +-
 .../client/solrj/cloud/autoscaling/TestPolicy.java | 272 ++++++++++++++++++++-
 .../solrj/cloud/autoscaling/TestPolicy2.java       |  43 ++++
 .../solrj/cloud/autoscaling/TestPolicy2Old.java}   |  11 +-
 .../solrj/cloud/autoscaling/TestPolicyOld.java}    |  10 +-
 10 files changed, 504 insertions(+), 68 deletions(-)
 copy solr/solrj/src/{java/org/apache/solr/client/solrj/cloud/autoscaling/package-info.java => test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2Old.java} (87%)
 copy solr/solrj/src/{java/org/apache/solr/client/solrj/cloud/autoscaling/package-info.java => test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicyOld.java} (88%)


[lucene-solr] 03/03: SOLR-13504: In autoscaling policies, use an explicit 'put : on-each' to specify the the rules is applied on each node (#694)

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 5c677f8f4688f9749dd22dccd3253bf0e2845861
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Mon Jun 3 18:13:18 2019 +1000

    SOLR-13504: In autoscaling policies, use an explicit 'put : on-each'   to specify the the rules is applied on each node (#694)
    
    SOLR-13504: In autoscaling policies, use an explicit 'put : on-each' to specify the the rules is applied on each node
---
 solr/CHANGES.txt                                   |   3 +
 .../cloud/autoscaling/AutoScalingHandlerTest.java  |  16 +++
 .../client/solrj/cloud/autoscaling/Clause.java     | 133 +++++++++++++++------
 .../client/solrj/cloud/autoscaling/Policy.java     |   2 +-
 .../client/solrj/cloud/autoscaling/Variable.java   |   5 +-
 .../solrj/cloud/autoscaling/VariableBase.java      |   2 +-
 .../client/solrj/cloud/autoscaling/TestPolicy.java |  21 ++--
 7 files changed, 132 insertions(+), 50 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 28d566e..306364a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -59,6 +59,9 @@ New Features
 * SOLR-13504: In autoscaling policies, use an explicit 'nodeset' attribute for filtering
   nodes instead of using them directly at the toplevel (noble)
 
+* SOLR-13504: In autoscaling policies, use an explicit 'put : on-each'
+  to specify the the rules is applied on each node  (noble)
+
 Bug Fixes
 ----------------------
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
index 5b98e44..f83c164 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
@@ -714,6 +714,22 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
     List clusterPolicy = (List) loaded.get("cluster-policy");
     assertNotNull(clusterPolicy);
     assertEquals(3, clusterPolicy.size());
+
+    setClusterPolicyCommand = "{" +
+        " 'set-cluster-policy': [" +
+        "      {'cores':'<10', 'node':'#ANY'}," +
+        "      {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
+        "      {'replica':0, put : on-each, nodeset:{'nodeRole':'overseer'} }" +
+        "    ]" +
+        "}";
+    req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+    data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+    loaded = ZkNodeProps.load(data);
+    clusterPolicy = (List) loaded.get("cluster-policy");
+    assertNotNull(clusterPolicy);
+    assertEquals(3, clusterPolicy.size());
   }
 
   @Test
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
index 382e5f3..adfbfcb 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
@@ -66,12 +66,14 @@ public class Clause implements MapWriter, Comparable<Clause> {
   boolean nodeSetPresent = false;
   Condition collection, shard, replica, tag, globalTag;
   final Replica.Type type;
+  Put put;
   boolean strict;
 
   protected Clause(Clause clause, Function<Condition, Object> computedValueEvaluator) {
     this.original = clause.original;
     this.hashCode = original.hashCode();
     this.type = clause.type;
+    this.put = clause.put;
     this.nodeSetPresent = clause.nodeSetPresent;
     this.collection = clause.collection;
     this.shard = clause.shard;
@@ -84,7 +86,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
   }
 
   // internal use only
-  Clause(Map<String, Object> original, Condition tag, Condition globalTag, boolean isStrict,  boolean nodeSetPresent) {
+  Clause(Map<String, Object> original, Condition tag, Condition globalTag, boolean isStrict, Put put, boolean nodeSetPresent) {
     this.hashCode = original.hashCode();
     this.original = original;
     this.tag = tag;
@@ -94,6 +96,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
     this.hasComputedValue = false;
     this.strict = isStrict;
     derivedFrom = null;
+    this.put = put;
     this.nodeSetPresent = nodeSetPresent;
   }
 
@@ -103,6 +106,12 @@ public class Clause implements MapWriter, Comparable<Clause> {
     this.hashCode = original.hashCode();
     String type = (String) m.get("type");
     this.type = type == null || ANY.equals(type) ? null : Replica.Type.valueOf(type.toUpperCase(Locale.ROOT));
+    String put = (String) m.getOrDefault("put", m.containsKey(NODESET)? Put.ON_ALL.val: null );
+    if (put != null) {
+      this.put = Put.get(put);
+      if (this.put == null) throwExp(m, "invalid value for put : {0}", put);
+    }
+
     strict = Boolean.parseBoolean(String.valueOf(m.getOrDefault("strict", "true")));
     Optional<String> globalTagName = m.keySet().stream().filter(Policy.GLOBAL_ONLY_TAGS::contains).findFirst();
     if (globalTagName.isPresent()) {
@@ -348,7 +357,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
     } catch (IllegalArgumentException iae) {
       throw iae;
     } catch (Exception e) {
-      throwExp(m, "Invalid tag : {0} ", s);
+      throwExp(m, " Invalid tag : {0} "+ e.getMessage(), s);
       return null;
     }
   }
@@ -400,26 +409,25 @@ public class Clause implements MapWriter, Comparable<Clause> {
     return operand;
   }
 
+
+  private boolean isRowPass(ComputedValueEvaluator eval, Object t, Row row) {
+    eval.node = row.node;
+    if (t instanceof Condition) {
+      Condition tag = (Condition) t;
+      if (tag.computedType != null) tag = evaluateValue(tag, eval);
+      return tag.isPass(row);
+    } else {
+      return t.equals(row.getVal(tag.name));
+    }
+  }
+
   List<Violation> testGroupNodes(Policy.Session session, double[] deviations) {
     //e.g:  {replica:'#EQUAL', shard:'#EACH',  sysprop.zone:'#EACH'}
     ComputedValueEvaluator eval = new ComputedValueEvaluator(session);
     eval.collName = (String) collection.getValue();
     Violation.Ctx ctx = new Violation.Ctx(this, session.matrix, eval);
 
-    Set tags = new HashSet();
-    for (Row row : session.matrix) {
-      eval.node = row.node;
-      Condition tag = this.tag;
-      if (tag.computedType != null) tag = evaluateValue(tag, eval);
-      Object val = row.getVal(tag.name);
-      if (val != null) {
-        if (tag.op == LESS_THAN || tag.op == GREATER_THAN) {
-          tags.add(this.tag);
-        } else if (tag.isPass(val)) {
-          tags.add(val);
-        }
-      }
-    }
+    Set tags = getUniqueTags(session, eval);
     if (tags.isEmpty()) return Collections.emptyList();
 
     Set<String> shards = getShardNames(session, eval);
@@ -428,17 +436,10 @@ public class Clause implements MapWriter, Comparable<Clause> {
       final ReplicaCount replicaCount = new ReplicaCount();
       eval.shardName = s;
 
-      for (Object t : tags) {
+      for (Object tag : tags) {
         replicaCount.reset();
         for (Row row : session.matrix) {
-          eval.node = row.node;
-          if (t instanceof Condition) {
-            Condition tag = (Condition) t;
-            if (tag.computedType != null) tag = evaluateValue(tag, eval);
-            if (!tag.isPass(row)) continue;
-          } else {
-            if (!t.equals(row.getVal(tag.name))) continue;
-          }
+         if(!isRowPass(eval, tag, row)) continue;
           addReplicaCountsForNode(eval, replicaCount, row);
         }
 
@@ -451,27 +452,57 @@ public class Clause implements MapWriter, Comparable<Clause> {
               null,
               replicaCountCopy,
               sealedClause.getReplica().replicaCountDelta(replicaCountCopy),
-              t);
-          ctx.resetAndAddViolation(t, replicaCountCopy, violation);
-          sealedClause.addViolatingReplicas(sealedClause.tag, eval, ctx, tag.name, t, violation, session);
+              tag);
+          ctx.resetAndAddViolation(tag, replicaCountCopy, violation);
+          sealedClause.addViolatingReplicasForGroup(sealedClause.tag, eval, ctx, this.tag.name, tag, violation, session.matrix);
           if (!this.strict && deviations != null) {
-            tag.varType.computeDeviation(session, deviations, replicaCount, sealedClause);
+            this.tag.varType.computeDeviation(session, deviations, replicaCount, sealedClause);
           }
         } else {
-          if (replica.op == RANGE_EQUAL) tag.varType.computeDeviation(session, deviations, replicaCount, sealedClause);
+          if (replica.op == RANGE_EQUAL) this.tag.varType.computeDeviation(session, deviations, replicaCount, sealedClause);
         }
       }
     }
     return ctx.allViolations;
   }
 
-  void addViolatingReplicas(Condition tag,
-                            ComputedValueEvaluator eval,
-                            Violation.Ctx ctx, String tagName, Object tagVal,
-                            Violation violation,
-                            Policy.Session session) {
+  private Set getUniqueTags(Policy.Session session, ComputedValueEvaluator eval) {
+    Set tags =  new HashSet();
+    if(tag.op == WILDCARD){
+      for (Row row : session.matrix) {
+        eval.node = row.node;
+        Condition tag = this.tag;
+        if (tag.computedType != null) tag = evaluateValue(tag, eval);
+        Object val = row.getVal(tag.name);
+        if (val != null) {
+          if (tag.op == LESS_THAN || tag.op == GREATER_THAN) {
+            tags.add(this.tag);
+          } else if (tag.isPass(val)) {
+            tags.add(val);
+          }
+        }
+      }
+
+    } else {
+
+      if (tag.op == LESS_THAN || tag.op == GREATER_THAN || tag.op == RANGE_EQUAL || tag.op == NOT_EQUAL) {
+        tags.add(tag); // eg: freedisk > 100
+      } else if (tag.val instanceof Collection) {
+        tags.addAll((Collection) tag.val); //e: sysprop.zone:[east,west]
+      } else {
+        tags.add(tag.val);//
+      }
+    }
+    return tags;
+  }
+
+  void addViolatingReplicasForGroup(Condition tag,
+                                    ComputedValueEvaluator eval,
+                                    Violation.Ctx ctx, String tagName, Object tagVal,
+                                    Violation violation,
+                                    List<Row> nodes) {
     if (tag.varType.addViolatingReplicas(ctx)) return;
-    for (Row row : session.matrix) {
+    for (Row row : nodes) {
       if (tagVal.equals(row.getVal(tagName))) {
         row.forEachReplica(eval.collName, ri -> {
           if (Policy.ANY.equals(eval.shardName)
@@ -535,7 +566,8 @@ public class Clause implements MapWriter, Comparable<Clause> {
               sealedClause.getReplica().replicaCountDelta(replicaCountCopy),
               eval.node);
           ctx.resetAndAddViolation(row.node, replicaCountCopy, violation);
-          sealedClause.addViolatingReplicas(sealedClause.tag, eval, ctx, NODE, row.node, violation, session);
+          sealedClause.addViolatingReplicasForGroup(sealedClause.tag, eval, ctx, NODE, row.node, violation,
+              Collections.singletonList(row));
           if (!this.strict && deviations != null) {
             tag.varType.computeDeviation(session, deviations, replicaCount, sealedClause);
           }
@@ -568,6 +600,14 @@ public class Clause implements MapWriter, Comparable<Clause> {
 
   public List<Violation> test(Policy.Session session, double[] deviations) {
     if (isPerCollectiontag()) {
+      if(nodeSetPresent) {
+        if(put == Put.ON_EACH){
+          return testPerNode(session, deviations) ;
+        } else {
+          return testGroupNodes(session, deviations);
+        }
+      }
+
       return tag.varType == Type.NODE ||
           (tag.varType.meta.isNodeSpecificVal() && replica.computedType == null) ?
           testPerNode(session, deviations) :
@@ -581,7 +621,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
         if (!sealedClause.getGlobalTag().isPass(r)) {
           ctx.resetAndAddViolation(r.node, null, new Violation(sealedClause, null, null, r.node, r.getVal(sealedClause.globalTag.name),
               sealedClause.globalTag.delta(r.getVal(globalTag.name)), r.node));
-          addViolatingReplicas(sealedClause.globalTag, computedValueEvaluator, ctx, Type.CORES.tagName, r.node, ctx.currentViolation, session);
+          addViolatingReplicasForGroup(sealedClause.globalTag, computedValueEvaluator, ctx, Type.CORES.tagName, r.node, ctx.currentViolation, session.matrix);
 
         }
       }
@@ -710,4 +750,21 @@ public class Clause implements MapWriter, Comparable<Clause> {
 
   public static final String METRICS_PREFIX = "metrics:";
 
+  enum Put {
+    ON_ALL("on-all"), ON_EACH("on-each");
+
+    public final String val;
+
+    Put(String s) {
+      this.val = s;
+    }
+
+    public static Put get(String s) {
+      for (Put put : values()) {
+        if (put.val.equals(s)) return put;
+      }
+      return null;
+    }
+  }
+
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
index c0ba31b..7b5c6c4 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
@@ -557,7 +557,7 @@ public class Policy implements MapWriter {
           if (!withCollMap.isEmpty()) {
             Clause withCollClause = new Clause((Map<String,Object>)Utils.fromJSONString("{withCollection:'*' , node: '#ANY'}") ,
                 new Condition(NODE.tagName, "#ANY", Operand.EQUAL, null, null),
-                new Condition(WITH_COLLECTION.tagName,"*" , Operand.EQUAL, null, null), true, false
+                new Condition(WITH_COLLECTION.tagName,"*" , Operand.EQUAL, null, null), true, null, false
             );
             expandedClauses.add(withCollClause);
           }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
index 72e571f..0f494e3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
@@ -214,8 +214,9 @@ public interface Variable {
     @Meta(name = "STRING",
         type = String.class,
         wildCards = Policy.EACH,
-        supportArrayVals = true)
-    STRING,
+        supportArrayVals = true
+    )
+    SYSPROP,
 
     @Meta(name = "node",
         type = String.class,
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java
index f6686ce..3001f11 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java
@@ -93,7 +93,7 @@ public class VariableBase implements Variable {
 
   public static Type getTagType(String name) {
     Type info = Type.get(name);
-    if (info == null && name.startsWith(ImplicitSnitch.SYSPROP)) info = Type.STRING;
+    if (info == null && name.startsWith(ImplicitSnitch.SYSPROP)) info = Type.SYSPROP;
     if (info == null && name.startsWith(Clause.METRICS_PREFIX)) info = Type.LAZY;
     return info;
   }
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 1cea205..dacfc37 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
@@ -701,7 +701,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
     expectThrows(IllegalArgumentException.class,
         () -> Clause.create("{cores: '>14%' , node:'#ANY'}"));
      clause  = Clause.create("{replica:1, nodeset : {sysprop.zone : east}}");
-     assertEquals(Variable.Type.STRING, clause.tag.varType);
+     assertEquals(Variable.Type.SYSPROP, clause.tag.varType);
      clause  =Clause.create("{replica:1, nodeset : [node1, node2, node3]}");
      assertEquals(Variable.Type.NODE, clause.tag.varType);
      assertEquals(Operand.IN, clause.tag.op);
@@ -719,7 +719,12 @@ public class TestPolicy extends SolrTestCaseJ4 {
         () -> Clause.create("{replica:1, nodeset : {sysprop.zone : east  , port: 8983 }}"));
     assertTrue(exp.getMessage().contains("nodeset must only have one and only one key"));
     clause = Clause.create("{'replica': '#ALL', 'nodeset': {'freedisk': '>700'}, 'strict': false}");
+    assertEquals(clause.put, Clause.Put.ON_ALL);
     assertEquals(Operand.GREATER_THAN , clause.tag.op);
+    clause = Clause.create("{'replica': '#ALL', put: on-each,  'nodeset': {sysprop.zone : east}}");
+    assertEquals(clause.put, Clause.Put.ON_EACH);
+    exp = expectThrows(IllegalArgumentException.class, ()-> Clause.create("{'replica': '#ALL', put: on-Each,  'nodeset': {sysprop.zone : east}}"));
+    assertTrue(exp.getMessage().contains("invalid value for put : on-Each"));
 
   }
 
@@ -1240,8 +1245,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
           "  'cluster-policy': [" +
           "    { 'replica': 0, nodeset : {'nodeRole': 'overseer'}}" +
           "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
-          "    { 'replica': 0, 'shard': '#EACH', nodeset : { sysprop.fs : '!ssd'},  type : TLOG }" +
-          "    { 'replica': 0, 'shard': '#EACH', nodeset : {sysprop.fs : '!slowdisk'} ,  type : PULL }" +
+          "    { 'replica': 0, 'shard': '#EACH',  nodeset : { sysprop.fs : '!ssd'},  type : TLOG }" +
+          "    { 'replica': 0, 'shard': '#EACH', put:'on-each' nodeset : {sysprop.fs : '!slowdisk'} ,  type : PULL }" +
           "  ]" +
           "}");
 
@@ -1366,8 +1371,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
           "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY', 'collection':'newColl'}," +
           "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY', 'collection':'newColl2', type : PULL}," +
           "    { 'replica': '<3', 'shard': '#EACH', 'node': '#ANY', 'collection':'newColl2'}," +
-          "    { 'replica': 0, 'shard': '#EACH', nodeset:{ sysprop.fs : '!ssd'},  type : TLOG }" +
-          "    { 'replica': 0, 'shard': '#EACH', nodeset : {sysprop.fs : '!slowdisk'} ,  type : PULL }" +
+          "    { 'replica': 0, 'shard': '#EACH', put: on-each , nodeset:{ sysprop.fs : '!ssd'},  type : TLOG }" +
+          "    { 'replica': 0, 'shard': '#EACH', put: on-each ,nodeset : {sysprop.fs : '!slowdisk'} ,  type : PULL }" +
           "  ]" +
           "}");
 
@@ -2389,8 +2394,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "  cluster-preferences :[{ minimize : cores, precision : 2 }]}";
     if(useNodeset){
       autoScalingjson = "  { cluster-policy:[" +
-          "    { replica :'0', nodeset:{ freedisk:'<1000'}}," +
-          "    { replica :0, nodeset : {nodeRole : overseer}}]," +
+          "    { replica :'0', put:on-each ,   nodeset:{ freedisk:'<1000'}}," +
+          "    { replica :0, put : on-each , nodeset : {nodeRole : overseer}}]," +
           "  cluster-preferences :[{ minimize : cores, precision : 2 }]}";
     }
     AutoScalingConfig cfg = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScalingjson));
@@ -2422,7 +2427,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
     if(useNodeset){
       autoScalingjson =  "  { cluster-policy:[" +
           "    { replica :'#ALL', nodeset:{ freedisk:'>1000'}}," +
-          "    { replica :0 , nodeset : {nodeRole : overseer}}]," +
+          "    { replica :0 , put: on-each , nodeset : {nodeRole : overseer}}]," +
           "  cluster-preferences :[{ minimize : cores, precision : 2 }]}";
     }
     cfg = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScalingjson));


[lucene-solr] 02/03: SOLR-13504 improve autoscaling syntax by adding a nodeset attribute (#693)

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit b3caaffd6e9fc665ab2b4f0cab2dfe7b83c477fe
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Sun Jun 2 18:54:28 2019 +1000

    SOLR-13504 improve autoscaling syntax by adding a nodeset attribute (#693)
    
    * SOLR-13504:  more checks and tests
---
 .../org/apache/solr/client/solrj/cloud/autoscaling/Clause.java   | 8 ++------
 .../apache/solr/client/solrj/cloud/autoscaling/VariableBase.java | 9 +++++++++
 .../apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java   | 8 +++++++-
 .../apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java  | 2 +-
 4 files changed, 19 insertions(+), 8 deletions(-)

diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
index d4ae968..382e5f3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
@@ -57,13 +57,13 @@ import static org.apache.solr.common.util.Utils.toJSONString;
  */
 public class Clause implements MapWriter, Comparable<Clause> {
   public static final String NODESET = "nodeset";
-  private static final Set<String> IGNORE_TAGS = new HashSet<>(Arrays.asList(REPLICA, COLLECTION, SHARD, "strict", "type", "put", NODESET));
+  static final Set<String> IGNORE_TAGS = new HashSet<>(Arrays.asList(REPLICA, COLLECTION, SHARD, "strict", "type", "put", NODESET));
 
   private final int hashCode;
   final boolean hasComputedValue;
   final Map<String, Object> original;
   final Clause derivedFrom;
-  private boolean nodeSetPresent = false;
+  boolean nodeSetPresent = false;
   Condition collection, shard, replica, tag, globalTag;
   final Replica.Type type;
   boolean strict;
@@ -568,10 +568,6 @@ public class Clause implements MapWriter, Comparable<Clause> {
 
   public List<Violation> test(Policy.Session session, double[] deviations) {
     if (isPerCollectiontag()) {
-      if(nodeSetPresent) {
-
-      }
-
       return tag.varType == Type.NODE ||
           (tag.varType.meta.isNodeSpecificVal() && replica.computedType == null) ?
           testPerNode(session, deviations) :
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java
index aaa874d..f6686ce 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java
@@ -45,6 +45,15 @@ public class VariableBase implements Variable {
     }
   }
 
+  @Override
+  public String postValidate(Condition condition) {
+    if(Clause.IGNORE_TAGS.contains(condition.getName())) return null;
+    if(condition.getOperand() == Operand.WILDCARD && condition.clause.nodeSetPresent){
+      return "#EACH not supported in tags in nodeset";
+    }
+    return null;
+  }
+
   static Object getOperandAdjustedValue(Object val, Object original) {
     if (original instanceof Condition) {
       Condition condition = (Condition) original;
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 9abbc54..1cea205 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
@@ -707,6 +707,12 @@ public class TestPolicy extends SolrTestCaseJ4 {
      assertEquals(Operand.IN, clause.tag.op);
 
     expectThrows(IllegalArgumentException.class,
+        () -> Clause.create("{replica:1, nodeset : {sysprop.zone : '#EACH'}}"));
+
+    expectThrows(IllegalArgumentException.class,
+        () -> Clause.create("{replica:1, nodeset : {host : '#EACH'}}"));
+
+    expectThrows(IllegalArgumentException.class,
         () -> Clause.create("{replica:1, node: n1, nodeset : {sysprop.zone : east}}"));
 
     IllegalArgumentException exp = expectThrows(IllegalArgumentException.class,
@@ -1912,7 +1918,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
           "{replica:0, nodeset:{'nodeRole':'overseer'},'strict':false}," +
           "{'replica':'<1','node':'node3'}," +
           "{'replica':'<2','node':'#ANY','shard':'#EACH'}," +
-          "{'replica':'<3','shard':'#EACH', nodeset : { 'sysprop.rack':'#EACH'}}" +
+          "{'replica':'<3','shard':'#EACH', nodeset : { 'sysprop.rack':[rack1, rack2, rack3, rack4]}}" +
           "]" +
           "}" +
           "}";
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 6ec31cd..2bceeab 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
@@ -87,7 +87,7 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
         "  'cluster-preferences':[{ minimize : cores},{maximize : freedisk, precision : 50}]}";
     if(useNodeset){
       autoScalingjson = "{cluster-policy:[" +
-          "    { replica : '<3' , shard : '#EACH', nodeset:{sysprop.zone: '#EACH'} } ]," +
+          "    { replica : '<3' , shard : '#EACH', nodeset:{sysprop.zone: [east , west]} } ]," +
           "  'cluster-preferences':[{ minimize : cores},{maximize : freedisk, precision : 50}]}";
     }
     policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoScalingjson));


[lucene-solr] 01/03: SOLR-13504: improve autoscaling syntax by adding a nodeset attribute (#691)

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit dbbab57398e37c88e1733a9dbd25e2d824d33455
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Fri May 31 17:45:55 2019 +1000

    SOLR-13504: improve autoscaling syntax by adding a nodeset attribute (#691)
    
    SOLR-13504: In autoscaling policies, use an explicit 'nodeset' attribute for filtering nodes instead of using them directly at the toplevel
---
 solr/CHANGES.txt                                   |   3 +
 .../client/solrj/cloud/autoscaling/Clause.java     |  71 ++++--
 .../client/solrj/cloud/autoscaling/Policy.java     |   2 +-
 .../client/solrj/cloud/autoscaling/TestPolicy.java | 261 ++++++++++++++++++++-
 .../solrj/cloud/autoscaling/TestPolicy2.java       |  43 ++++
 .../solrj/cloud/autoscaling/TestPolicy2Old.java    |  26 ++
 .../solrj/cloud/autoscaling/TestPolicyOld.java     |  25 ++
 7 files changed, 410 insertions(+), 21 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b184e6c..28d566e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -56,6 +56,9 @@ New Features
 
 * SOLR-13494: Add DeepRandomStream implementation (Joel Bernstein)
 
+* SOLR-13504: In autoscaling policies, use an explicit 'nodeset' attribute for filtering
+  nodes instead of using them directly at the toplevel (noble)
+
 Bug Fixes
 ----------------------
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
index d23cbec..d4ae968 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
@@ -56,12 +56,14 @@ import static org.apache.solr.common.util.Utils.toJSONString;
  * Represents a set of conditions in the policy
  */
 public class Clause implements MapWriter, Comparable<Clause> {
-  private static final Set<String> IGNORE_TAGS = new HashSet<>(Arrays.asList(REPLICA, COLLECTION, SHARD, "strict", "type"));
+  public static final String NODESET = "nodeset";
+  private static final Set<String> IGNORE_TAGS = new HashSet<>(Arrays.asList(REPLICA, COLLECTION, SHARD, "strict", "type", "put", NODESET));
 
   private final int hashCode;
   final boolean hasComputedValue;
   final Map<String, Object> original;
   final Clause derivedFrom;
+  private boolean nodeSetPresent = false;
   Condition collection, shard, replica, tag, globalTag;
   final Replica.Type type;
   boolean strict;
@@ -70,6 +72,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
     this.original = clause.original;
     this.hashCode = original.hashCode();
     this.type = clause.type;
+    this.nodeSetPresent = clause.nodeSetPresent;
     this.collection = clause.collection;
     this.shard = clause.shard;
     this.tag = evaluateValue(clause.tag, computedValueEvaluator);
@@ -81,7 +84,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
   }
 
   // internal use only
-  Clause(Map<String, Object> original, Condition tag, Condition globalTag, boolean isStrict)  {
+  Clause(Map<String, Object> original, Condition tag, Condition globalTag, boolean isStrict,  boolean nodeSetPresent) {
     this.hashCode = original.hashCode();
     this.original = original;
     this.tag = tag;
@@ -91,6 +94,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
     this.hasComputedValue = false;
     this.strict = isStrict;
     derivedFrom = null;
+    this.nodeSetPresent = nodeSetPresent;
   }
 
   private Clause(Map<String, Object> m) {
@@ -117,7 +121,9 @@ public class Clause implements MapWriter, Comparable<Clause> {
       }
       this.replica = parse(REPLICA, m);
       if (replica.op == WILDCARD) throw new IllegalArgumentException("replica val cannot be null" + toJSONString(m));
-      m.forEach(this::parseCondition);
+
+      this.nodeSetPresent = parseNodeset(m);
+      m.forEach((s, o) -> parseCondition(s, o, m));
     }
     if (tag == null)
       throw new RuntimeException("Invalid op, must have one and only one tag other than collection, shard,replica " + toJSONString(m));
@@ -131,9 +137,33 @@ public class Clause implements MapWriter, Comparable<Clause> {
     hasComputedValue = hasComputedValue();
   }
 
+  private boolean parseNodeset(Map<String, Object> m) {
+    if (!m.containsKey(NODESET)) return false;
+    Object o = m.get(NODESET);
+    if (o instanceof Map) {
+      Map map = (Map) o;
+      if (map.size() != 1) {
+        throwExp(m, "nodeset must only have one and only one key");
+      }
+      String key = (String) map.keySet().iterator().next();
+      parseCondition(key, o, m);
+    } else if (o instanceof List) {
+      List l = (List) o;
+      for (Object it : l) {
+        if (it instanceof String) continue;
+        else throwExp(m, "nodeset :[]must have only string values");
+      }
+      parseCondition("node", o, m);
+    } else {
+      throwExp(m, "invalid value for nodeset, must be an object or a list of String");
+    }
+    return true;
+  }
+
   public Condition getThirdTag() {
     return globalTag == null ? tag : globalTag;
   }
+
   private void doPostValidate(Condition... conditions) {
     for (Condition condition : conditions) {
       if (condition == null) continue;
@@ -198,12 +228,12 @@ public class Clause implements MapWriter, Comparable<Clause> {
     return globalTag == null;
   }
 
-  void parseCondition(String s, Object o) {
+  void parseCondition(String s, Object o, Map m) {
     if (IGNORE_TAGS.contains(s)) return;
     if (tag != null) {
-      throw new IllegalArgumentException("Only one tag other than collection, shard, replica is possible");
+      throwExp(m, "Only one tag other than collection, shard, replica is possible");
     }
-    tag = parse(s, singletonMap(s, o));
+    tag = parse(s, o instanceof Map? (Map<String, Object>) o : singletonMap(s, o));
   }
 
   private int compareTypes(Replica.Type t1, Replica.Type t2) {
@@ -249,8 +279,8 @@ public class Clause implements MapWriter, Comparable<Clause> {
   public boolean equals(Object o) {
     if (this == o) return true;
     if (!(o instanceof Clause)) return false;
-    Clause that = (Clause)o;
-    return  Objects.equals(this.original, that.original);
+    Clause that = (Clause) o;
+    return Objects.equals(this.original, that.original);
   }
 
   //replica value is zero
@@ -265,14 +295,14 @@ public class Clause implements MapWriter, Comparable<Clause> {
         new SealedClause(this, computedValueEvaluator);
   }
 
-  Condition parse(String s, Map<String,Object> m) {
+  Condition parse(String s, Map<String, Object> m) {
 
     Object expectedVal = null;
     ComputedType computedType = null;
     Object val = m.get(s);
     Type varType = VariableBase.getTagType(s);
     if (varType.meta.isHidden()) {
-      throwExp(m,"''{0}'' is not allowed", varType.tagName);
+      throwExp(m, "''{0}'' is not allowed", varType.tagName);
     }
     try {
       String conditionName = s.trim();
@@ -297,12 +327,12 @@ public class Clause implements MapWriter, Comparable<Clause> {
             computedType = t;
             strVal = changedVal;
             if (varType == null || !varType.supportedComputedTypes.contains(computedType)) {
-              throwExp(m,"''{0}'' is not allowed for variable :  ''{1}''",t,conditionName);
+              throwExp(m, "''{0}'' is not allowed for variable :  ''{1}''", t, conditionName);
             }
           }
         }
         if (computedType == null && ((String) val).charAt(0) == '#' && !varType.wildCards.contains(val)) {
-          throwExp(m, "''{0}'' is not an allowed value for ''{1}'', supported value is : {2} ", val, conditionName,  varType.wildCards );
+          throwExp(m, "''{0}'' is not an allowed value for ''{1}'', supported value is : {2} ", val, conditionName, varType.wildCards);
 
         }
         operand = varType == null ? operand : varType.getOperand(operand, strVal, computedType);
@@ -318,16 +348,16 @@ public class Clause implements MapWriter, Comparable<Clause> {
     } catch (IllegalArgumentException iae) {
       throw iae;
     } catch (Exception e) {
-      throwExp(m, "Invalid tag : {0} ",s );
+      throwExp(m, "Invalid tag : {0} ", s);
       return null;
     }
   }
 
-  public void throwExp(Map<String, Object> clause, String msg, Object... args) {
-    throw new IllegalArgumentException("syntax error in clause :"+ toJSONString(clause)+ " , msg:  "+  formatString(msg, args));
+  public static void throwExp(Map clause, String msg, Object... args) {
+    throw new IllegalArgumentException("syntax error in clause :" + toJSONString(clause) + " , msg:  " + formatString(msg, args));
   }
 
-  private List readListVal(Map m, List val, Type varType, String conditionName) {
+  private static List readListVal(Map m, List val, Type varType, String conditionName) {
     List list = val;
     list = (List) list.stream()
         .map(it -> varType.validate(conditionName, it, true))
@@ -360,7 +390,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
     return list;
   }
 
-  private Operand getOperand(String strVal) {
+  private static Operand getOperand(String strVal) {
     Operand operand;
     if (Policy.ANY.equals(strVal) || Policy.EACH.equals(strVal)) operand = WILDCARD;
     else if (strVal.startsWith(NOT_EQUAL.operand)) operand = NOT_EQUAL;
@@ -456,6 +486,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
   public static long addReplicaCountsForNode = 0;
   public static long addReplicaCountsForNodeCacheMiss = 0;
   public static final String PERSHARD_REPLICAS = Clause.class.getSimpleName() + ".perShardReplicas";
+
   private void addReplicaCountsForNode(ComputedValueEvaluator computedValueEvaluator, ReplicaCount replicaCount, Row node) {
     addReplicaCountsForNode++;
 
@@ -537,6 +568,10 @@ public class Clause implements MapWriter, Comparable<Clause> {
 
   public List<Violation> test(Policy.Session session, double[] deviations) {
     if (isPerCollectiontag()) {
+      if(nodeSetPresent) {
+
+      }
+
       return tag.varType == Type.NODE ||
           (tag.varType.meta.isNodeSpecificVal() && replica.computedType == null) ?
           testPerNode(session, deviations) :
@@ -618,7 +653,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
    * @param isRuleVal is this provided in the rule
    * @return actual validated value
    */
-  public static Object  validate(String name, Object val, boolean isRuleVal) {
+  public static Object validate(String name, Object val, boolean isRuleVal) {
     if (val == null) return null;
     Type info = VariableBase.getTagType(name);
     if (info == null) throw new RuntimeException("Unknown type :" + name);
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
index 7c48831..c0ba31b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
@@ -557,7 +557,7 @@ public class Policy implements MapWriter {
           if (!withCollMap.isEmpty()) {
             Clause withCollClause = new Clause((Map<String,Object>)Utils.fromJSONString("{withCollection:'*' , node: '#ANY'}") ,
                 new Condition(NODE.tagName, "#ANY", Operand.EQUAL, null, null),
-                new Condition(WITH_COLLECTION.tagName,"*" , Operand.EQUAL, null, null), true
+                new Condition(WITH_COLLECTION.tagName,"*" , Operand.EQUAL, null, null), true, false
             );
             expandedClauses.add(withCollClause);
           }
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 c46eadf..9abbc54 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
@@ -86,8 +86,12 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.AD
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
 
 public class TestPolicy extends SolrTestCaseJ4 {
+  boolean useNodeset ;
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  public TestPolicy(){
+    useNodeset = true;
+  }
   static Suggester createSuggester(SolrCloudManager cloudManager, Map jsonObj, Suggester seed) throws IOException, InterruptedException {
     Policy.Session session = null;
     if (seed != null) session = seed.session;
@@ -696,6 +700,20 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
     expectThrows(IllegalArgumentException.class,
         () -> Clause.create("{cores: '>14%' , node:'#ANY'}"));
+     clause  = Clause.create("{replica:1, nodeset : {sysprop.zone : east}}");
+     assertEquals(Variable.Type.STRING, clause.tag.varType);
+     clause  =Clause.create("{replica:1, nodeset : [node1, node2, node3]}");
+     assertEquals(Variable.Type.NODE, clause.tag.varType);
+     assertEquals(Operand.IN, clause.tag.op);
+
+    expectThrows(IllegalArgumentException.class,
+        () -> Clause.create("{replica:1, node: n1, nodeset : {sysprop.zone : east}}"));
+
+    IllegalArgumentException exp = expectThrows(IllegalArgumentException.class,
+        () -> Clause.create("{replica:1, nodeset : {sysprop.zone : east  , port: 8983 }}"));
+    assertTrue(exp.getMessage().contains("nodeset must only have one and only one key"));
+    clause = Clause.create("{'replica': '#ALL', 'nodeset': {'freedisk': '>700'}, 'strict': false}");
+    assertEquals(Operand.GREATER_THAN , clause.tag.op);
 
   }
 
@@ -976,6 +994,19 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
         "  ]" +
         "}");
+
+    if(useNodeset) {
+      policies = (Map) Utils.fromJSONString("{" +
+          "  'cluster-preferences': [" +
+          "    { 'maximize': 'freedisk', 'precision': 50}," +
+          "    { 'minimize': 'cores', 'precision': 50}" +
+          "  ]," +
+          "  'cluster-policy': [" +
+          "    { 'replica': 0, nodeset:{'nodeRole': 'overseer'}}" +
+          "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
+          "  ]" +
+          "}");
+    }
     AutoScalingConfig config = new AutoScalingConfig(policies);
     Policy policy = config.getPolicy();
     Policy.Session session = policy.createSession(provider);
@@ -1011,6 +1042,19 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { 'replica': '<3', 'shard': '#EACH', 'node': '#ANY'}," +
         "  ]" +
         "}");
+    if(useNodeset){
+      policies = (Map) Utils.fromJSONString("{" +
+          "  'cluster-preferences': [" +
+          "    { 'maximize': 'freedisk', 'precision': 50}," +
+          "    { 'minimize': 'cores', 'precision': 50}" +
+          "  ]," +
+          "  'cluster-policy': [" +
+          "    { 'replica': 0, nodeset: {'nodeRole': 'overseer'}}" +
+          "    { 'replica': '<3', 'shard': '#EACH', 'node': '#ANY'}," +
+          "  ]" +
+          "}");
+
+    }
     config = new AutoScalingConfig(policies);
     policy = config.getPolicy();
     session = policy.createSession(provider);
@@ -1055,6 +1099,20 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { 'replica': '<3', 'shard': 'shard2', 'node': '#ANY'}," +
         "  ]" +
         "}");
+    if(useNodeset){
+      policies = (Map) Utils.fromJSONString("{" +
+          "  'cluster-preferences': [" +
+          "    { 'maximize': 'freedisk', 'precision': 50}," +
+          "    { 'minimize': 'cores', 'precision': 50}" +
+          "  ]," +
+          "  'cluster-policy': [" +
+          "    { 'replica': 0, nodeset:{'nodeRole': 'overseer'}}" +
+          "    { 'replica': '<2', 'shard': 'shard1', 'node': '#ANY'}," +
+          "    { 'replica': '<3', 'shard': 'shard2', 'node': '#ANY'}," +
+          "  ]" +
+          "}");
+
+    }
     config = new AutoScalingConfig(policies);
     policy = config.getPolicy();
     session = policy.createSession(provider);
@@ -1167,6 +1225,21 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { 'replica': 0, 'shard': '#EACH', sysprop.fs : '!slowdisk' ,  type : PULL }" +
         "  ]" +
         "}");
+    if(useNodeset){
+      policies = (Map) Utils.fromJSONString("{" +
+          "  'cluster-preferences': [" +
+          "    { 'maximize': 'freedisk', 'precision': 50}," +
+          "    { 'minimize': 'cores', 'precision': 50}" +
+          "  ]," +
+          "  'cluster-policy': [" +
+          "    { 'replica': 0, nodeset : {'nodeRole': 'overseer'}}" +
+          "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
+          "    { 'replica': 0, 'shard': '#EACH', nodeset : { sysprop.fs : '!ssd'},  type : TLOG }" +
+          "    { 'replica': 0, 'shard': '#EACH', nodeset : {sysprop.fs : '!slowdisk'} ,  type : PULL }" +
+          "  ]" +
+          "}");
+
+    }
     Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
         "node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4, sysprop.fs: slowdisk}," +
         "node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3, sysprop.fs: unknown }," +
@@ -1276,6 +1349,23 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { 'replica': 0, 'shard': '#EACH', sysprop.fs : '!slowdisk' ,  type : PULL }" +
         "  ]" +
         "}");
+    if(useNodeset){
+      policies = (Map) Utils.fromJSONString("{" +
+          "  'cluster-preferences': [" +
+          "    { 'maximize': 'freedisk', 'precision': 50}," +
+          "    { 'minimize': 'cores', 'precision': 1}" +
+          "  ]," +
+          "  'cluster-policy': [" +
+          "    { 'replica': 0, nodeset : {'nodeRole': 'overseer'}}" +
+          "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY', 'collection':'newColl'}," +
+          "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY', 'collection':'newColl2', type : PULL}," +
+          "    { 'replica': '<3', 'shard': '#EACH', 'node': '#ANY', 'collection':'newColl2'}," +
+          "    { 'replica': 0, 'shard': '#EACH', nodeset:{ sysprop.fs : '!ssd'},  type : TLOG }" +
+          "    { 'replica': 0, 'shard': '#EACH', nodeset : {sysprop.fs : '!slowdisk'} ,  type : PULL }" +
+          "  ]" +
+          "}");
+
+    }
     Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
         "node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4, sysprop.fs: slowdisk}," +
         "node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3, sysprop.fs: unknown}," +
@@ -1400,6 +1490,26 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    ]" +
         "  }" +
         "}");
+    if(useNodeset){
+      map = (Map) Utils.fromJSONString("{" +
+          "  'cluster-preferences': [" +
+          "    { 'maximize': 'freedisk', 'precision': 50}," +
+          "    { 'minimize': 'cores', 'precision': 50}" +
+          "  ]," +
+          "  'cluster-policy': [" +
+          "    { 'replica': 0, nodeset: {'nodeRole': 'overseer'}}," +
+          "    { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}" +
+          "  ]," +
+          "  'policies': {" +
+          "    'policy1': [" +
+          "      { 'replica': '1', nodeset:{ 'sysprop.fs': 'ssd'}, 'shard': '#EACH'}," +
+          "      { 'replica': '<2', 'shard': '#ANY', 'node': '#ANY'}," +
+          "      { 'replica': '<2', 'shard': '#EACH',nodeset:{ 'sysprop.rack': 'rack1'}}" +
+          "    ]" +
+          "  }" +
+          "}");
+
+    }
     Policy policy = new Policy(map);
     List<Clause> clauses = Policy.mergePolicies("mycoll", policy.getPolicies().get("policy1"), policy.getClusterPolicy());
     Collections.sort(clauses);
@@ -1407,8 +1517,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
     assertEquals("1", String.valueOf(clauses.get(0).original.get("replica")));
     assertEquals("0", String.valueOf(clauses.get(1).original.get("replica")));
     assertEquals("#ANY", clauses.get(3).original.get("shard"));
-    assertEquals("rack1", clauses.get(2).original.get("sysprop.rack"));
-    assertEquals("overseer", clauses.get(1).original.get("nodeRole"));
+    assertEquals("rack1", clauses.get(2).tag.val);
+    assertEquals("overseer",clauses.get(1).tag.val);
   }
 
   public void testConditionsSort() {
@@ -1419,6 +1529,16 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "      { 'replica':'<2', 'node':'#ANY', 'shard':'#EACH'}," +
         "      { 'replica':1, 'sysprop.rack':'rack1'}]" +
         "  }";
+    if(useNodeset){
+      rules = "{" +
+          "    'cluster-policy':[" +
+          "      { 'nodeRole':'overseer', replica: 0,  'strict':false}," +
+          "      { 'replica':'<1', 'node':'node3', 'shard':'#EACH'}," +
+          "      { 'replica':'<2', 'node':'#ANY', 'shard':'#EACH'}," +
+          "      { 'replica':1, nodeset: {'sysprop.rack':'rack1'}}]" +
+          "  }";
+
+    }
     Policy p = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
     List<Clause> clauses = new ArrayList<>(p.getClusterPolicy());
     Collections.sort(clauses);
@@ -1436,6 +1556,17 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "{minimize:cores , precision:2}," +
         "{maximize:freedisk, precision:50}, " +
         "{minimize:heapUsage, precision:1000}]}";
+    if(useNodeset){
+      rules = "{" +
+          "cluster-policy:[" +
+          "{nodeset:{nodeRole:'overseer'},replica : 0 , strict:false}," +
+          "{replica:'<1',node:node3}," +
+          "{replica:'<2',node:'#ANY', shard:'#EACH'}]," +
+          " cluster-preferences:[" +
+          "{minimize:cores , precision:2}," +
+          "{maximize:freedisk, precision:50}, " +
+          "{minimize:heapUsage, precision:1000}]}";
+    }
 
     Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
         "node1:{cores:12, freedisk: 334, heapUsage:10480}," +
@@ -1624,6 +1755,18 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "      'cluster-preferences':[" +
         "      {'minimize':'cores', 'precision':3}," +
         "      {'maximize':'freedisk','precision':100}]}";
+    if(useNodeset){
+      autoscaleJson = "{" +
+          "      'cluster-policy':[" +
+          "      {'cores':'<10','node':'#ANY'}," +
+          "      {'replica':'<3','shard':'#EACH','node':'#ANY'}," +
+          "      { 'replica': 2, nodeset: {'sysprop.fs': 'ssd'}, 'shard': '#EACH'}," +//greedy condition
+          "      {nodeset:{'nodeRole':'overseer'},'replica':'0'}]," +
+          "      'cluster-preferences':[" +
+          "      {'minimize':'cores', 'precision':3}," +
+          "      {'maximize':'freedisk','precision':100}]}";
+
+    }
     Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
         "node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4}," +
         "node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3}," +
@@ -1667,6 +1810,17 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "      {'minimize':'cores', 'precision':3}," +
         "      {'maximize':'freedisk','precision':100}]}";
 
+    if(useNodeset){
+      autoscaleJson = "{" +
+          "      'cluster-policy':[" +
+          "      {'cores':'<10','node':'#ANY'}," +
+          "      {'replica':'<3','shard':'#EACH','node':'#ANY'}," +
+          "      {nodeset: {'nodeRole':'overseer'},'replica':'0'}]," +
+          "      'cluster-preferences':[" +
+          "      {'minimize':'cores', 'precision':3}," +
+          "      {'maximize':'freedisk','precision':100}]}";
+
+    }
 
     Map replicaInfoMap = (Map) Utils.fromJSONString("{ '127.0.0.1:60099_solr':{}," +
         " '127.0.0.1:60089_solr':{'compute_plan_action_test':{'shard1':[" +
@@ -1741,6 +1895,31 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "}" +
         "}";
 
+    if(useNodeset){
+      rules = "{" +
+          "'cluster-preferences':[" +
+          "{'minimize':'cores','precision':2}," +
+          "{'maximize':'freedisk','precision':50}," +
+          "{'minimize':'heapUsage','precision':1000}" +
+          "]," +
+          "'cluster-policy':[" +
+          "{replica:0, nodeset:{'nodeRole':'overseer'},'strict':false}," +
+          "{'replica':'<1','node':'node3'}," +
+          "{'replica':'<2','node':'#ANY','shard':'#EACH'}" +
+          "]," +
+          "'policies':{" +
+          "'p1':[" +
+          "{replica:0, nodeset:{'nodeRole':'overseer'},'strict':false}," +
+          "{'replica':'<1','node':'node3'}," +
+          "{'replica':'<2','node':'#ANY','shard':'#EACH'}," +
+          "{'replica':'<3','shard':'#EACH', nodeset : { 'sysprop.rack':'#EACH'}}" +
+          "]" +
+          "}" +
+          "}";
+
+
+    }
+
     Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
         "node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4}," +
         "node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3}," +
@@ -1928,6 +2107,26 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    ]" +
         "  }" +
         "}";
+    if(useNodeset){
+      autoScaleJson = "{" +
+          "  cluster-preferences: [" +
+          "    { maximize : freedisk , precision: 50}," +
+          "    { minimize : cores, precision: 2}" +
+          "  ]," +
+          "  cluster-policy: [" +
+          "    { replica : '0' , nodeset: {nodeRole: overseer}}," +
+          "    { replica: '<2', shard: '#ANY', node: '#ANY'" +
+          "    }" +
+          "  ]," +
+          "  policies: {" +
+          "    policy1: [" +
+          "      { replica: '<2', shard: '#EACH', node: '#ANY'}," +
+          "      { replica: '<2', shard: '#EACH', nodeset:{ sysprop.rack: rack1}}" +
+          "    ]" +
+          "  }" +
+          "}";
+
+    }
 
 
     Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
@@ -2003,6 +2202,15 @@ public class TestPolicy extends SolrTestCaseJ4 {
         " cluster-policy:[{cores:'<10',node:'#ANY'}," +
         "       {replica:'<2', shard:'#EACH',node:'#ANY'}," +
         "       { nodeRole:overseer,replica:0}]}";
+    if(useNodeset){
+      autoScalingjson = "cluster-preferences:[" +
+          "       {minimize : cores}," +
+          "       {'maximize':freedisk , precision:100}],    " +
+          " cluster-policy:[{cores:'<10',node:'#ANY'}," +
+          "       {replica:'<2', shard:'#EACH',node:'#ANY'}," +
+          "       {nodeset:{ nodeRole:overseer},replica:0}]}";
+
+    }
     Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoScalingjson));
     Policy.Session session = policy.createSession(cloudManagerWithData((Map) loadFromResource("testComputePlanAfterNodeAdded.json")));
     Suggester suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
@@ -2018,6 +2226,14 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { replica :'<2',  node:'#ANY'}," +
         "    { nodeRole : overseer, replica :0}]," +
         "  cluster-preferences :[{ minimize : cores }]}";
+    if(useNodeset){
+      autoScalingjson = "  { cluster-policy:[" +
+          "    { cores :'<10', node :'#ANY'}," +
+          "    { replica :'<2',  node:'#ANY'}," +
+          "    { nodeset:{nodeRole : overseer}, replica :0}]," +
+          "  cluster-preferences :[{ minimize : cores }]}";
+
+    }
     List<Suggester.SuggestionInfo> l = PolicyHelper.getSuggestions(new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScalingjson)),
         cloudManagerWithData((Map) loadFromResource("testReplicaCountSuggestions.json")));
     assertFalse(l.isEmpty());
@@ -2123,6 +2339,12 @@ public class TestPolicy extends SolrTestCaseJ4 {
   public void testFreeDiskDeviation() {
     Map map = (Map) loadFromResource("testFreeDiskDeviation.json");
     AutoScalingConfig cfg = new AutoScalingConfig((Map<String, Object>) map.get("config"));
+    if(useNodeset){
+      cfg = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString("{" +
+          "    'cluster-policy': [{'replica':'<2', 'shard':'#EACH', 'node':'#ANY'}," +
+          "      {'replica': '#ALL', 'nodeset': {'freedisk': '>700'}, 'strict': false}]" +
+          "  }"));
+    }
     SolrCloudManager scm = cloudManagerWithData(map);
     Suggester suggester = cfg.getPolicy()
         .createSession(scm)
@@ -2159,6 +2381,12 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { replica :'0', freedisk:'<1000'}," +
         "    { nodeRole : overseer, replica :0}]," +
         "  cluster-preferences :[{ minimize : cores, precision : 2 }]}";
+    if(useNodeset){
+      autoScalingjson = "  { cluster-policy:[" +
+          "    { replica :'0', nodeset:{ freedisk:'<1000'}}," +
+          "    { replica :0, nodeset : {nodeRole : overseer}}]," +
+          "  cluster-preferences :[{ minimize : cores, precision : 2 }]}";
+    }
     AutoScalingConfig cfg = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScalingjson));
     List<Violation> violations = cfg.getPolicy().createSession(cloudManagerWithData((Map) loadFromResource("testFreeDiskSuggestions.json"))).getViolations();
     assertEquals(1, violations.size());
@@ -2185,6 +2413,12 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { replica :'#ALL', freedisk:'>1000'}," +
         "    { nodeRole : overseer, replica :0}]," +
         "  cluster-preferences :[{ minimize : cores, precision : 2 }]}";
+    if(useNodeset){
+      autoScalingjson =  "  { cluster-policy:[" +
+          "    { replica :'#ALL', nodeset:{ freedisk:'>1000'}}," +
+          "    { replica :0 , nodeset : {nodeRole : overseer}}]," +
+          "  cluster-preferences :[{ minimize : cores, precision : 2 }]}";
+    }
     cfg = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScalingjson));
     violations = cfg.getPolicy().createSession(cloudManagerWithData((Map) loadFromResource("testFreeDiskSuggestions.json"))).getViolations();
     assertEquals(1, violations.size());
@@ -2237,6 +2471,17 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    { 'replica': '1', shard:'#EACH', sysprop.fs : 'ssd'}" +
         "  ]" +
         "}";
+    if(useNodeset){
+      autoScalingjson = "{" +
+          "  'cluster-preferences': [" +
+          "    { 'maximize': 'freedisk', 'precision': 50}," +
+          "    { 'minimize': 'cores', 'precision': 3}" +
+          "  ]," +
+          "  'cluster-policy': [" +
+          "    { 'replica': '1', shard:'#EACH', nodeset:{ sysprop.fs : 'ssd'}}" +
+          "  ]" +
+          "}";
+    }
 
 
     AutoScalingConfig cfg = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScalingjson));
@@ -2260,6 +2505,18 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "  ]" +
         "}";
 
+    if(useNodeset){
+      autoScalingjson = "{" +
+          "  'cluster-preferences': [" +
+          "    { 'maximize': 'freedisk', 'precision': 50}," +
+          "    { 'minimize': 'cores', 'precision': 3}" +
+          "  ]," +
+          "  'cluster-policy': [" +
+          "    { 'replica': 0, shard:'#EACH', nodeset :{ port : '8983'}}" +
+          "  ]" +
+          "}";
+
+    }
     AutoScalingConfig cfg = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScalingjson));
     List<Violation> violations = cfg.getPolicy().createSession(cloudManagerWithData((Map) loadFromResource("testPortSuggestions.json"))).getViolations();
     assertEquals(2, violations.size());
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 982b017..6ec31cd 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
@@ -53,6 +53,11 @@ import static org.apache.solr.common.util.Utils.MAPOBJBUILDER;
 import static org.apache.solr.common.util.Utils.getObjectByPath;
 
 public class TestPolicy2 extends SolrTestCaseJ4 {
+  boolean useNodeset ;
+  public TestPolicy2(){
+    useNodeset = true;
+  }
+
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public void testEqualOnNonNode() {
@@ -60,6 +65,12 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
     String autoScalingjson = "{cluster-policy:[" +
         "    { replica : '<3' , shard : '#EACH', sysprop.zone: [east,west] } ]," +
         "  'cluster-preferences':[{ minimize : cores},{maximize : freedisk, precision : 50}]}";
+    if(useNodeset){
+      autoScalingjson = "{cluster-policy:[" +
+          "    { replica : '<3' , shard : '#EACH', nodeset:{ sysprop.zone: [east,west] }} ]," +
+          "  'cluster-preferences':[{ minimize : cores},{maximize : freedisk, precision : 50}]}";
+      
+    }
     Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoScalingjson));
     Policy.Session session = policy.createSession(createCloudManager(l.get(0), l.get(1)));
     List<Violation> violations = session.getViolations();
@@ -74,6 +85,11 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
     autoScalingjson = "{cluster-policy:[" +
         "    { replica : '<3' , shard : '#EACH', sysprop.zone: '#EACH' } ]," +
         "  'cluster-preferences':[{ minimize : cores},{maximize : freedisk, precision : 50}]}";
+    if(useNodeset){
+      autoScalingjson = "{cluster-policy:[" +
+          "    { replica : '<3' , shard : '#EACH', nodeset:{sysprop.zone: '#EACH'} } ]," +
+          "  'cluster-preferences':[{ minimize : cores},{maximize : freedisk, precision : 50}]}";
+    }
     policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoScalingjson));
     session = policy.createSession(createCloudManager(l.get(0), l.get(1)));
     violations = session.getViolations();
@@ -335,6 +351,22 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
     Map<String, Object> m = (Map<String, Object>) loadFromResource("testSysPropSuggestions.json");
 
     Map<String, Object> conf = (Map<String, Object>) getObjectByPath(m, false, "diagnostics/config");
+    if(useNodeset){
+      conf = (Map<String, Object>) Utils.fromJSONString("{" +
+          "    'cluster-preferences':[{" +
+          "      'minimize':'cores'," +
+          "      'precision':1}," +
+          "      {" +
+          "        'maximize':'freedisk'," +
+          "        'precision':100}," +
+          "      {" +
+          "        'minimize':'sysLoadAvg'," +
+          "        'precision':10}]," +
+          "    'cluster-policy':[{" +
+          "      'replica':'<3'," +
+          "      'shard':'#EACH'," +
+          "      nodeset: {'sysprop.zone':['east','west']}}]}");
+    }
     Policy policy = new Policy(conf);
     SolrCloudManager cloudManagerFromDiagnostics = createCloudManagerFromDiagnostics(m);
     Policy.Session session = policy.createSession(cloudManagerFromDiagnostics);
@@ -361,6 +393,17 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
         "      {'minimize':'sysLoadAvg','precision':10}]," +
         "    'cluster-policy':[" +
         "{'replica':'<5','shard':'#EACH','sysprop.zone':['east','west']}]}";
+    if(useNodeset){
+      conf = " {" +
+          "    'cluster-preferences':[{" +
+          "      'minimize':'cores'," +
+          "      'precision':1}," +
+          "      {'maximize':'freedisk','precision':100}," +
+          "      {'minimize':'sysLoadAvg','precision':10}]," +
+          "    'cluster-policy':[" +
+          "{'replica':'<5','shard':'#EACH', nodeset:{'sysprop.zone':['east','west']}}]}";
+
+    }
     Map<String, Object> m = (Map<String, Object>) loadFromResource("testSuggestionsRebalanceOnly.json");
     SolrCloudManager cloudManagerFromDiagnostics = createCloudManagerFromDiagnostics(m);
     AutoScalingConfig autoScalingConfig = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(conf));
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2Old.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2Old.java
new file mode 100644
index 0000000..8b411d6
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2Old.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.cloud.autoscaling;
+
+public class TestPolicy2Old extends TestPolicy2 {
+  public TestPolicy2Old(){
+    super();
+    useNodeset = false;
+  }
+
+}
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicyOld.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicyOld.java
new file mode 100644
index 0000000..44cdbf0
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicyOld.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.cloud.autoscaling;
+
+public class TestPolicyOld extends TestPolicy {
+  public TestPolicyOld(){
+    super();
+    useNodeset = false;
+  }
+}