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:38 UTC

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

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;
+  }
+}