You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2018/08/04 06:34:29 UTC

lucene-solr:master: SOLR-12592: added support for range , percentage and decimal

Repository: lucene-solr
Updated Branches:
  refs/heads/master f8db5d0af -> 664187f73


SOLR-12592: added support for range , percentage and decimal


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

Branch: refs/heads/master
Commit: 664187f73af51059c75d51206bf53900433f669c
Parents: f8db5d0
Author: noble <no...@apache.org>
Authored: Sat Aug 4 16:34:23 2018 +1000
Committer: noble <no...@apache.org>
Committed: Sat Aug 4 16:34:23 2018 +1000

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +-
 ...olrcloud-autoscaling-policy-preferences.adoc |   7 +-
 .../client/solrj/cloud/autoscaling/Clause.java  | 178 +------------------
 .../solrj/cloud/autoscaling/ComputedType.java   |  99 +++++++++++
 .../solrj/cloud/autoscaling/Condition.java      | 122 +++++++++++++
 .../solrj/cloud/autoscaling/CoresVariable.java  |  50 ++++--
 .../cloud/autoscaling/FreeDiskVariable.java     |   6 +-
 .../client/solrj/cloud/autoscaling/Operand.java |   6 +-
 .../client/solrj/cloud/autoscaling/Policy.java  |   4 +-
 .../cloud/autoscaling/ReplicaVariable.java      |  24 +--
 .../solrj/cloud/autoscaling/Variable.java       |  31 ++--
 .../solrj/cloud/autoscaling/VariableBase.java   |   8 +-
 .../solrj/cloud/autoscaling/Violation.java      |   4 +-
 .../solrj/cloud/autoscaling/TestPolicy.java     |  63 ++++---
 14 files changed, 347 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 314d045..3d9c68c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -138,7 +138,7 @@ New Features
 
 * SOLR-12402: Factor out SolrDefaultStreamFactory class. (Christine Poerschke)
 
-* SOLR-12592: support #EQUAL function in cores in autoscaling policies (noble)
+* SOLR-12592: support #EQUAL function, range operator, decimal and percentage in cores in autoscaling policies (noble)
 
 Bug Fixes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
index c3f8612..9641424 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
@@ -90,7 +90,12 @@ A policy is a hard rule to be satisfied by each node. If a node does not satisfy
 A policy can have the following attributes:
 
 `cores`::
-This is a special attribute that applies to the entire cluster. It can only be used along with the `node` attribute and no other. This attribute is optional.
+This is a special attribute that applies to the entire cluster. It can only be used along with the `node` attribute and no other. The value of this attribute can be
+* a positive integer . e.g : "`3`"
+* a number with a decimal value . e.g: "`1.66`" . This means both 1 and 2 are acceptable values but the system would prefer `2`
+* a number range. Such as `"3-5"` . This means `3,4,5` are acceptable values
+* a percentage value . e.g: `33%` . This is computed to a decimal value at runtime
+* `#EQUAL` : Divide the no:of cores equally among all the nodes or a subset of nodes
 
 `collection`::
 The name of the collection to which the policy rule should apply. If omitted, the rule applies to all collections. This attribute is optional.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
----------------------------------------------------------------------
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 5fe6894..cd9212b 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
@@ -251,6 +251,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
   }
 
   Condition parse(String s, Map m) {
+
     Object expectedVal = null;
     ComputedType computedType = null;
     Object val = m.get(s);
@@ -461,183 +462,6 @@ public class Clause implements MapWriter, Comparable<Clause> {
     return false;
   }
 
-  public enum ComputedType {
-    NULL(),
-    EQUAL() {
-      @Override
-      public String wrap(String value) {
-        return "#EQUAL";
-      }
-
-      @Override
-      public String match(String val) {
-        if ("#EQUAL".equals(val)) return "1";
-        return null;
-      }
-
-    },
-    ALL() {
-      @Override
-      public String wrap(String value) {
-        return "#ALL";
-      }
-
-      @Override
-      public String match(String val) {
-        if ("#ALL".equals(val)) return "1";
-        return null;
-      }
-
-    },
-    PERCENT {
-      @Override
-      public String wrap(String value) {
-        return value + "%";
-      }
-
-      @Override
-      public String match(String val) {
-        if (val != null && !val.isEmpty() && val.charAt(val.length() - 1) == '%') {
-          String newVal = val.substring(0, val.length() - 1);
-          double d;
-          try {
-            d = Double.parseDouble(newVal);
-          } catch (NumberFormatException e) {
-            throw new IllegalArgumentException("Invalid percentage value : " + val);
-          }
-          if (d < 0 || d > 100) {
-            throw new IllegalArgumentException("Percentage value must lie between [1 -100] : provided value : " + val);
-          }
-          return newVal;
-        } else {
-          return null;
-        }
-      }
-
-      @Override
-      public Object compute(Object val, Condition c) {
-        if (val == null || Clause.parseDouble(c.name, val) == 0) return 0d;
-        return Clause.parseDouble(c.name, val) * Clause.parseDouble(c.name, c.val).doubleValue() / 100;
-      }
-
-      @Override
-      public String toString() {
-        return "%";
-      }
-    };
-
-    // return null if there is no match. return a modified string
-    // if there is a match
-    public String match(String val) {
-      return null;
-    }
-
-    public String wrap(String value) {
-      return value;
-    }
-
-    public Object compute(Object val, Condition c) {
-      return val;
-    }
-
-  }
-
-  public static class Condition implements MapWriter {
-    final String name;
-    final Object val;
-    final Type varType;
-    final ComputedType computedType;
-    final Operand op;
-    private Clause clause;
-
-    Condition(String name, Object val, Operand op, ComputedType computedType, Clause parent) {
-      this.name = name;
-      this.val = val;
-      this.op = op;
-      varType = VariableBase.getTagType(name);
-      this.computedType = computedType;
-      this.clause = parent;
-    }
-
-    @Override
-    public void writeMap(EntryWriter ew) throws IOException {
-      String value = op.wrap(val);
-      if (computedType != null) value = computedType.wrap(value);
-      ew.put(name, value);
-    }
-
-    @Override
-    public String toString() {
-      return jsonStr();
-    }
-
-    public Clause getClause() {
-      return clause;
-    }
-
-    boolean isPass(Object inputVal) {
-      return isPass(inputVal, null);
-    }
-
-    boolean isPass(Object inputVal, Row row) {
-      if (computedType != null) {
-        throw new IllegalStateException("This is supposed to be called only from a Condition with no computed value or a SealedCondition");
-
-      }
-      if (inputVal instanceof ReplicaCount) inputVal = ((ReplicaCount) inputVal).getVal(getClause().type);
-      return varType.match(inputVal, op, val, name, row);
-    }
-
-
-    boolean isPass(Row row) {
-      return isPass(row.getVal(name), row);
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that instanceof Condition) {
-        Condition c = (Condition) that;
-        return Objects.equals(c.name, name) && Objects.equals(c.val, val) && c.op == op;
-      }
-      return false;
-    }
-
-    public Double delta(Object val) {
-      if (val instanceof ReplicaCount) val = ((ReplicaCount) val).getVal(getClause().type);
-      if (this.val instanceof String) {
-        if (op == LESS_THAN || op == GREATER_THAN) {
-          return op
-              .opposite(getClause().isReplicaZero() && this == getClause().tag)
-              .delta(Clause.parseDouble(name, this.val), Clause.parseDouble(name, val));
-        } else {
-          return 0d;
-        }
-      } else {
-        if (this == getClause().getReplica()) {
-          Double delta = op.delta(this.val, val);
-          return getClause().isReplicaZero() ? -1 * delta : delta;
-        } else {
-          return op
-              .opposite(getClause().isReplicaZero() && this == getClause().getTag())
-              .delta(this.val, val);
-        }
-
-      }
-    }
-
-    public String getName() {
-      return name;
-    }
-
-    public Object getValue() {
-      return val;
-    }
-
-    public Operand getOperand() {
-      return op;
-    }
-  }
-
   public boolean isStrict() {
     return strict;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ComputedType.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ComputedType.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ComputedType.java
new file mode 100644
index 0000000..73bca1f
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ComputedType.java
@@ -0,0 +1,99 @@
+/*
+ * 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 enum ComputedType {
+  NULL(),
+  EQUAL() {
+    @Override
+    public String wrap(String value) {
+      return "#EQUAL";
+    }
+
+    @Override
+    public String match(String val) {
+      if ("#EQUAL".equals(val)) return "1";
+      return null;
+    }
+
+  },
+  ALL() {
+    @Override
+    public String wrap(String value) {
+      return "#ALL";
+    }
+
+    @Override
+    public String match(String val) {
+      if ("#ALL".equals(val)) return "1";
+      return null;
+    }
+
+  },
+  PERCENT {
+    @Override
+    public String wrap(String value) {
+      return value + "%";
+    }
+
+    @Override
+    public String match(String val) {
+      if (val != null && !val.isEmpty() && val.charAt(val.length() - 1) == '%') {
+        String newVal = val.substring(0, val.length() - 1);
+        double d;
+        try {
+          d = Double.parseDouble(newVal);
+        } catch (NumberFormatException e) {
+          throw new IllegalArgumentException("Invalid percentage value : " + val);
+        }
+        if (d < 0 || d > 100) {
+          throw new IllegalArgumentException("Percentage value must lie between [1 -100] : provided value : " + val);
+        }
+        return newVal;
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public Object compute(Object val, Condition c) {
+      if (val == null || Clause.parseDouble(c.name, val) == 0) return 0d;
+      return Clause.parseDouble(c.name, val) * Clause.parseDouble(c.name, c.val).doubleValue() / 100;
+    }
+
+    @Override
+    public String toString() {
+      return "%";
+    }
+  };
+
+  // return null if there is no match. return a modified string
+  // if there is a match
+  public String match(String val) {
+    return null;
+  }
+
+  public String wrap(String value) {
+    return value;
+  }
+
+  public Object compute(Object val, Condition c) {
+    return val;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Condition.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Condition.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Condition.java
new file mode 100644
index 0000000..3a58804
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Condition.java
@@ -0,0 +1,122 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.solr.common.MapWriter;
+
+import static org.apache.solr.client.solrj.cloud.autoscaling.Operand.GREATER_THAN;
+import static org.apache.solr.client.solrj.cloud.autoscaling.Operand.LESS_THAN;
+
+public class Condition implements MapWriter {
+  final String name;
+  final Object val;
+  final Variable.Type varType;
+  final ComputedType computedType;
+  final Operand op;
+  Clause clause;
+
+  Condition(String name, Object val, Operand op, ComputedType computedType, Clause parent) {
+    this.name = name;
+    this.val = val;
+    this.op = op;
+    varType = VariableBase.getTagType(name);
+    this.computedType = computedType;
+    this.clause = parent;
+  }
+
+  @Override
+  public void writeMap(EntryWriter ew) throws IOException {
+    String value = op.wrap(val);
+    if (computedType != null) value = computedType.wrap(value);
+    ew.put(name, value);
+  }
+
+  @Override
+  public String toString() {
+    return jsonStr();
+  }
+
+  public Clause getClause() {
+    return clause;
+  }
+
+  boolean isPass(Object inputVal) {
+    return isPass(inputVal, null);
+  }
+
+  boolean isPass(Object inputVal, Row row) {
+    if (computedType != null) {
+      throw new IllegalStateException("This is supposed to be called only from a Condition with no computed value or a SealedCondition");
+
+    }
+    if (inputVal instanceof ReplicaCount) inputVal = ((ReplicaCount) inputVal).getVal(getClause().type);
+    return varType.match(inputVal, op, val, name, row);
+  }
+
+
+  boolean isPass(Row row) {
+    return isPass(row.getVal(name), row);
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that instanceof Condition) {
+      Condition c = (Condition) that;
+      return Objects.equals(c.name, name) && Objects.equals(c.val, val) && c.op == op;
+    }
+    return false;
+  }
+
+  public Double delta(Object val) {
+    if (val instanceof ReplicaCount) val = ((ReplicaCount) val).getVal(getClause().type);
+    if (this.val instanceof String) {
+      if (op == LESS_THAN || op == GREATER_THAN) {
+        return op
+            .opposite(getClause().isReplicaZero() && this == getClause().tag)
+            .delta(Clause.parseDouble(name, this.val), Clause.parseDouble(name, val));
+      } else {
+        return 0d;
+      }
+    } else {
+      if (this == getClause().getReplica()) {
+        Double delta = op.delta(this.val, val);
+        return getClause().isReplicaZero() ? -1 * delta : delta;
+      } else {
+        return op
+            .opposite(getClause().isReplicaZero() && this == getClause().getTag())
+            .delta(this.val, val);
+      }
+
+    }
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Object getValue() {
+    return val;
+  }
+
+  public Operand getOperand() {
+    return op;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java
index 45f9eb7..4df394b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java
@@ -17,6 +17,8 @@
 
 package org.apache.solr.client.solrj.cloud.autoscaling;
 
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
@@ -67,33 +69,47 @@ public class CoresVariable extends VariableBase {
   }
 
   @Override
-  public Object computeValue(Policy.Session session, Clause.Condition condition, String collection, String shard, String node) {
-    if (condition.computedType == Clause.ComputedType.EQUAL) {
-      int[] coresCount = new int[1];
-      int[] liveNodes = new int[1];
-      for (Row row : session.matrix) {
-        if (!row.isLive) continue;
-        liveNodes[0]++;
-        row.forEachReplica(replicaInfo -> coresCount[0]++);
-      }
-      return liveNodes[0] == 0 || coresCount[0] == 0 ? 0d : (double) coresCount[0] / (double) liveNodes[0];
+  public Object computeValue(Policy.Session session, Condition condition, String collection, String shard, String node) {
+    if (condition.computedType == ComputedType.EQUAL) {
+      AtomicInteger liveNodes = new AtomicInteger(0);
+      int coresCount = getTotalCores(session, liveNodes);
+      int numBuckets = condition.clause.tag.op == Operand.IN ?
+          ((Collection) condition.clause.tag.val).size() :
+          liveNodes.get();
+      return numBuckets == 0 || coresCount == 0 ? 0d : (double) coresCount / (double) numBuckets;
+    } else if (condition.computedType == ComputedType.PERCENT) {
+      return ComputedType.PERCENT.compute(getTotalCores(session, new AtomicInteger()), condition);
     } else {
       throw new IllegalArgumentException("Invalid computed type in " + condition);
     }
   }
 
+  private int getTotalCores(Policy.Session session, AtomicInteger liveNodes) {
+    int[] coresCount = new int[1];
+    for (Row row : session.matrix) {
+      if (!row.isLive) continue;
+      liveNodes.incrementAndGet();
+      row.forEachReplica(replicaInfo -> coresCount[0]++);
+    }
+    return coresCount[0];
+  }
+
   @Override
-  public String postValidate(Clause.Condition condition) {
-    Clause.Condition nodeTag = condition.getClause().getTag();
-    if (nodeTag.name.equals("node") && nodeTag.op == Operand.WILDCARD) {
-      return null;
-    } else {
-      throw new IllegalArgumentException("cores: '#EQUAL' can be used only with node: '#ANY'");
+  public String postValidate(Condition condition) {
+    Condition nodeTag = condition.getClause().getTag();
+    if (nodeTag.varType != Type.NODE) return "'cores' attribute can only be used with 'node' attribute";
+    if (condition.computedType == ComputedType.EQUAL) {
+      if (nodeTag.name.equals("node") && (nodeTag.op == Operand.WILDCARD || nodeTag.op == Operand.IN)) {
+        return null;
+      } else {
+        return "cores: '#EQUAL' can be used only with node: '#ANY', node :[....]";
+      }
     }
+    return null;
   }
 
   @Override
-  public Operand getOperand(Operand expected, Object strVal, Clause.ComputedType computedType) {
+  public Operand getOperand(Operand expected, Object strVal, ComputedType computedType) {
     return ReplicaVariable.checkForRangeOperand(expected, strVal, computedType);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java
index b598207..600a708 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java
@@ -46,11 +46,11 @@ public class FreeDiskVariable extends VariableBase {
   }
 
   @Override
-  public Object computeValue(Policy.Session session, Clause.Condition condition, String collection, String shard, String node) {
-    if (condition.computedType == Clause.ComputedType.PERCENT) {
+  public Object computeValue(Policy.Session session, Condition condition, String collection, String shard, String node) {
+    if (condition.computedType == ComputedType.PERCENT) {
       Row r = session.getNode(node);
       if (r == null) return 0d;
-      return Clause.ComputedType.PERCENT.compute(r.getVal(TOTALDISK.tagName), condition);
+      return ComputedType.PERCENT.compute(r.getVal(TOTALDISK.tagName), condition);
     }
     throw new IllegalArgumentException("Unsupported type " + condition.computedType);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Operand.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Operand.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Operand.java
index 3e2368c..d4835b8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Operand.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Operand.java
@@ -49,7 +49,7 @@ public enum Operand {
     }
 
     @Override
-    public Object readRuleValue(Clause.Condition condition) {
+    public Object readRuleValue(Condition condition) {
       if (condition.val instanceof String) {
         String strVal = ((String) condition.val).trim();
         int hyphenIdx = strVal.indexOf('-');
@@ -91,7 +91,7 @@ public enum Operand {
     }
 
     @Override
-    public Object readRuleValue(Clause.Condition condition) {
+    public Object readRuleValue(Condition condition) {
       return RANGE_EQUAL.readRuleValue(condition);
     }
 
@@ -202,7 +202,7 @@ public enum Operand {
     return operand + val.toString();
   }
 
-  public Object readRuleValue(Clause.Condition condition) {
+  public Object readRuleValue(Condition condition) {
     return condition.val;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Policy.java
----------------------------------------------------------------------
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 09aa244..b398caf 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
@@ -514,8 +514,8 @@ public class Policy implements MapWriter {
           Map<String, String> withCollMap = (Map<String, String>) vals.get("withCollection");
           if (!withCollMap.isEmpty()) {
             Clause withCollClause = new Clause((Map<String,Object>)Utils.fromJSONString("{withCollection:'*' , node: '#ANY'}") ,
-                new Clause.Condition(NODE.tagName, "#ANY", Operand.EQUAL, null, null),
-                new Clause.Condition(WITH_COLLECTION.tagName,"*" , Operand.EQUAL, null, null), true
+                new Condition(NODE.tagName, "#ANY", Operand.EQUAL, null, null),
+                new Condition(WITH_COLLECTION.tagName,"*" , Operand.EQUAL, null, null), true
             );
             expandedClauses.add(withCollClause);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaVariable.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaVariable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaVariable.java
index ab0a03a..6239ec3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaVariable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaVariable.java
@@ -30,7 +30,7 @@ class ReplicaVariable extends VariableBase {
     super(type);
   }
 
-  static int getRelevantReplicasCount(Policy.Session session, Clause.Condition cv, String collection, String shard) {
+  static int getRelevantReplicasCount(Policy.Session session, Condition cv, String collection, String shard) {
     AtomicInteger totalReplicasOfInterest = new AtomicInteger(0);
     Clause clause = cv.getClause();
     for (Row row : session.matrix) {
@@ -50,12 +50,12 @@ class ReplicaVariable extends VariableBase {
 
 
   @Override
-  public Operand getOperand(Operand expected, Object strVal, Clause.ComputedType computedType) {
-    if (computedType == Clause.ComputedType.ALL) return expected;
+  public Operand getOperand(Operand expected, Object strVal, ComputedType computedType) {
+    if (computedType == ComputedType.ALL) return expected;
     return checkForRangeOperand(expected, strVal, computedType);
   }
 
-  static Operand checkForRangeOperand(Operand expected, Object strVal, Clause.ComputedType computedType) {
+  static Operand checkForRangeOperand(Operand expected, Object strVal, ComputedType computedType) {
     if (strVal instanceof String) {
       String s = ((String) strVal).trim();
       int hyphenIdx = s.indexOf('-');
@@ -79,8 +79,8 @@ class ReplicaVariable extends VariableBase {
   }
 
   @Override
-  public String postValidate(Clause.Condition condition) {
-    if (condition.computedType == Clause.ComputedType.EQUAL) {
+  public String postValidate(Condition condition) {
+    if (condition.computedType == ComputedType.EQUAL) {
       if (condition.getClause().tag != null &&
 //              condition.getClause().tag.varType == NODE &&
           (condition.getClause().tag.op == Operand.WILDCARD || condition.getClause().tag.op == Operand.IN)) {
@@ -89,7 +89,7 @@ class ReplicaVariable extends VariableBase {
         return "'replica': '#EQUAL` must be used with 'node':'#ANY'";
       }
     }
-    if (condition.computedType == Clause.ComputedType.ALL) {
+    if (condition.computedType == ComputedType.ALL) {
       if (condition.getClause().tag != null && (condition.getClause().getTag().op == Operand.IN ||
           condition.getClause().getTag().op == Operand.WILDCARD)) {
         return StrUtils.formatString("array value or wild card cannot be used for tag {0} with replica : '#ALL'",
@@ -100,16 +100,16 @@ class ReplicaVariable extends VariableBase {
   }
 
   @Override
-  public Object computeValue(Policy.Session session, Clause.Condition cv, String collection, String shard, String node) {
-    if (cv.computedType == Clause.ComputedType.ALL)
+  public Object computeValue(Policy.Session session, Condition cv, String collection, String shard, String node) {
+    if (cv.computedType == ComputedType.ALL)
       return Double.valueOf(getRelevantReplicasCount(session, cv, collection, shard));
-    if (cv.computedType == Clause.ComputedType.EQUAL) {
+    if (cv.computedType == ComputedType.EQUAL) {
       int relevantReplicasCount = getRelevantReplicasCount(session, cv, collection, shard);
       double bucketsCount = getNumBuckets(session, cv.getClause());
       if (relevantReplicasCount == 0 || bucketsCount == 0) return 0;
       return (double) relevantReplicasCount / bucketsCount;
-    } else if (cv.computedType == Clause.ComputedType.PERCENT) {
-      return Clause.ComputedType.PERCENT.compute(getRelevantReplicasCount(session, cv, collection, shard), cv);
+    } else if (cv.computedType == ComputedType.PERCENT) {
+      return ComputedType.PERCENT.compute(getRelevantReplicasCount(session, cv, collection, shard), cv);
     } else {
       throw new IllegalArgumentException("Unsupported type " + cv.computedType);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
----------------------------------------------------------------------
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 c3d8ca2..d817c2f 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
@@ -42,6 +42,7 @@ public interface Variable {
   default boolean match(Object inputVal, Operand op, Object val, String name, Row row) {
     return op.match(val, validate(name, inputVal, false)) == Clause.TestStatus.PASS;
   }
+
   default Object convertVal(Object val) {
     return val;
   }
@@ -56,9 +57,9 @@ public interface Variable {
     }
   }
 
-  void getSuggestions(Suggestion.Ctx ctx) ;
+  void getSuggestions(Suggestion.Ctx ctx);
 
-  default Object computeValue(Policy.Session session, Clause.Condition condition, String collection, String shard, String node) {
+  default Object computeValue(Policy.Session session, Condition condition, String collection, String shard, String node) {
     return condition.val;
   }
 
@@ -67,11 +68,11 @@ public interface Variable {
   default void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
   }
 
-  default String postValidate(Clause.Condition condition) {
+  default String postValidate(Condition condition) {
     return null;
   }
 
-  default Operand getOperand(Operand expected, Object strVal, Clause.ComputedType computedType) {
+  default Operand getOperand(Operand expected, Object strVal, ComputedType computedType) {
     return expected;
   }
 
@@ -97,7 +98,7 @@ public interface Variable {
         type = Double.class,
         min = 0, max = -1,
         implementation = ReplicaVariable.class,
-        computedValues = {Clause.ComputedType.EQUAL, Clause.ComputedType.PERCENT, Clause.ComputedType.ALL})
+        computedValues = {ComputedType.EQUAL, ComputedType.PERCENT, ComputedType.ALL})
     REPLICA,
     @Meta(name = ImplicitSnitch.PORT,
         type = Long.class,
@@ -142,7 +143,7 @@ public interface Variable {
         associatedPerReplicaValue = Variable.coreidxsize,
         associatedPerNodeValue = "totaldisk",
         implementation = FreeDiskVariable.class,
-        computedValues = Clause.ComputedType.PERCENT)
+        computedValues = ComputedType.PERCENT)
     FREEDISK,
 
     @Meta(name = "totaldisk",
@@ -166,7 +167,7 @@ public interface Variable {
     @Meta(name = ImplicitSnitch.CORES,
         type = Double.class,
         min = 0, max = -1,
-        computedValues = Clause.ComputedType.EQUAL,
+        computedValues = {ComputedType.EQUAL, ComputedType.PERCENT},
         implementation = CoresVariable.class)
     CORES,
 
@@ -225,7 +226,7 @@ public interface Variable {
     public final String perReplicaValue;
     public final Set<String> associatedPerNodeValues;
     public final String metricsAttribute;
-    public final Set<Clause.ComputedType> supportedComputedTypes;
+    public final Set<ComputedType> supportedComputedTypes;
     final Variable impl;
 
 
@@ -238,7 +239,7 @@ public interface Variable {
       } catch (NoSuchFieldException e) {
         //cannot happen
       }
-      impl= VariableBase.loadImpl(meta, this);
+      impl = VariableBase.loadImpl(meta, this);
 
       this.tagName = meta.name();
       this.type = meta.type();
@@ -250,7 +251,7 @@ public interface Variable {
       this.associatedPerNodeValues = readSet(meta.associatedPerNodeValue());
       this.additive = meta.isAdditive();
       this.metricsAttribute = readStr(meta.metricsKey());
-      this.supportedComputedTypes = meta.computedValues()[0] == Clause.ComputedType.NULL ?
+      this.supportedComputedTypes = meta.computedValues()[0] == ComputedType.NULL ?
           emptySet() :
           unmodifiableSet(new HashSet(Arrays.asList(meta.computedValues())));
       this.wildCards = readSet(meta.wildCards());
@@ -282,10 +283,10 @@ public interface Variable {
 
     @Override
     public void addViolatingReplicas(Violation.Ctx ctx) {
-        impl.addViolatingReplicas(ctx);
+      impl.addViolatingReplicas(ctx);
     }
 
-    public Operand getOperand(Operand expected, Object val, Clause.ComputedType computedType) {
+    public Operand getOperand(Operand expected, Object val, ComputedType computedType) {
       return impl.getOperand(expected, val, computedType);
     }
 
@@ -294,7 +295,7 @@ public interface Variable {
       return impl.convertVal(val);
     }
 
-    public String postValidate(Clause.Condition condition) {
+    public String postValidate(Condition condition) {
       return impl.postValidate(condition);
     }
 
@@ -319,7 +320,7 @@ public interface Variable {
     }
 
     @Override
-    public Object computeValue(Policy.Session session, Clause.Condition condition, String collection, String shard, String node) {
+    public Object computeValue(Policy.Session session, Condition condition, String collection, String shard, String node) {
       return impl.computeValue(session, condition, collection, shard, node);
     }
 
@@ -360,6 +361,6 @@ public interface Variable {
 
     Class implementation() default void.class;
 
-    Clause.ComputedType[] computedValues() default Clause.ComputedType.NULL;
+    ComputedType[] computedValues() default ComputedType.NULL;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/VariableBase.java
----------------------------------------------------------------------
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 ad2b43b..8b3f1e1 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
@@ -40,8 +40,8 @@ public class VariableBase implements Variable {
   }
 
   static Object getOperandAdjustedValue(Object val, Object original) {
-    if (original instanceof Clause.Condition) {
-      Clause.Condition condition = (Clause.Condition) original;
+    if (original instanceof Condition) {
+      Condition condition = (Condition) original;
       if (condition.computedType == null && isIntegerEquivalent(val)) {
         if (condition.op == Operand.LESS_THAN) {
           //replica : '<3'
@@ -102,8 +102,8 @@ public class VariableBase implements Variable {
 
   @Override
   public Object validate(String name, Object val, boolean isRuleVal) {
-    if (val instanceof Clause.Condition) {
-      Clause.Condition condition = (Clause.Condition) val;
+    if (val instanceof Condition) {
+      Condition condition = (Condition) val;
       val = condition.op.readRuleValue(condition);
       if (val != condition.val) return val;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java
index 2f81291..39b3c0b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java
@@ -170,7 +170,7 @@ public class Violation implements MapWriter {
   }
 
   static class Ctx {
-    final Function<Clause.Condition, Object> evaluator;
+    final Function<Condition, Object> evaluator;
     String tagKey;
     Clause clause;
     ReplicaCount count;
@@ -178,7 +178,7 @@ public class Violation implements MapWriter {
     List<Row> allRows;
     List<Violation> allViolations = new ArrayList<>();
 
-    public Ctx(Clause clause, List<Row> allRows, Function<Clause.Condition, Object> evaluator) {
+    public Ctx(Clause clause, List<Row> allRows, Function<Condition, Object> evaluator) {
       this.allRows = allRows;
       this.clause = clause;
       this.evaluator = evaluator;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/664187f7/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
index 48d8f2e..3bd58aa 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
@@ -567,7 +567,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
     for (Suggester.SuggestionInfo suggestionInfo : l) {
       Map s = suggestionInfo.toMap(new LinkedHashMap<>());
       assertEquals("POST", Utils.getObjectByPath(s, true, "operation/method"));
-      if (Utils.getObjectByPath(s, false, "operation/command/add-replica") != null)  {
+      if (Utils.getObjectByPath(s, false, "operation/command/add-replica") != null) {
         numAdds++;
         assertEquals(1.0d, Utils.getObjectByPath(s, true, "violation/violation/delta"));
         assertEquals("/c/articles_coll/shards", Utils.getObjectByPath(s, true, "operation/path"));
@@ -577,7 +577,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
         assertEquals("/c/articles_coll", Utils.getObjectByPath(s, true, "operation/path"));
         targetNodes.add((String) Utils.getObjectByPath(s, true, "operation/command/move-replica/targetNode"));
         movedReplicas.add((String) Utils.getObjectByPath(s, true, "operation/command/move-replica/replica"));
-      } else  {
+      } else {
         fail("Unexpected operation type suggested for suggestion: " + suggestionInfo);
       }
     }
@@ -712,7 +712,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
     suggester.hint(Hint.SRC_NODE, "node1");
     SolrRequest op = suggester.getSuggestion();
     assertNotNull(op);
-    assertEquals("node2 should have been selected by move replica","node2",
+    assertEquals("node2 should have been selected by move replica", "node2",
         op.getParams().get("targetNode"));
 
     session = suggester.getSession();
@@ -801,13 +801,13 @@ public class TestPolicy extends SolrTestCaseJ4 {
       }
       throw new RuntimeException("");
     });
-    assertTrue( clause.getReplica().isPass(2));
+    assertTrue(clause.getReplica().isPass(2));
 
     clause = Clause.create("{replica: '3 - 5', node:'#ANY'}");
-    assertEquals(Operand.RANGE_EQUAL,  clause.getReplica().getOperand());
+    assertEquals(Operand.RANGE_EQUAL, clause.getReplica().getOperand());
     RangeVal range = (RangeVal) clause.getReplica().getValue();
-    assertEquals(3.0 , range.min);
-    assertEquals(5.0 , range.max);
+    assertEquals(3.0, range.min);
+    assertEquals(5.0, range.max);
     assertTrue(clause.replica.isPass(3));
     assertTrue(clause.replica.isPass(4));
     assertTrue(clause.replica.isPass(5));
@@ -832,7 +832,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
     expectThrows(IllegalArgumentException.class,
         () -> Clause.create("{replica: '#EQUAL', node:'node_1'}"));
     clause = Clause.create("{replica : 0, freedisk:'<20%'}");
-    assertEquals(clause.tag.computedType, Clause.ComputedType.PERCENT);
+    assertEquals(clause.tag.computedType, ComputedType.PERCENT);
     assertEquals(clause.tag.op, Operand.LESS_THAN);
     expectThrows(IllegalArgumentException.class,
         () -> Clause.create("{replica : 0, INDEX.sizeInGB:'>300'}"));
@@ -864,16 +864,39 @@ public class TestPolicy extends SolrTestCaseJ4 {
     expectThrows(IllegalArgumentException.class,
         () -> Clause.create("{replica: '#ALL' , shard: '#EACH' , sysprop.zone:'#EACH'}"));
     clause = Clause.create("{replica: '#EQUAL' , shard: '#EACH' , sysprop.zone:[east, west]}");
-    assertEquals(Clause.ComputedType.EQUAL, clause.replica.computedType);
+    assertEquals(ComputedType.EQUAL, clause.replica.computedType);
     assertEquals(Operand.IN, clause.tag.op);
     expectThrows(IllegalArgumentException.class,
         () -> Clause.create("{replica: '#EQUAL' , shard: '#EACH' , sysprop.zone:[east]}"));
 
     clause = Clause.create("{cores: '#EQUAL' , node:'#ANY'}");
-    assertEquals(Clause.ComputedType.EQUAL, clause.globalTag.computedType);
+    assertEquals(ComputedType.EQUAL, clause.globalTag.computedType);
     expectThrows(IllegalArgumentException.class,
         () -> Clause.create("{cores: '#EQUAL' , node:'node1'}"));
 
+    clause = Clause.create("{cores: '#EQUAL' , node:[node1 , node2 , node3]}");
+    assertEquals(Operand.IN, clause.getTag().op);
+    assertEquals(ComputedType.EQUAL, clause.getGlobalTag().computedType);
+
+    clause = Clause.create("{cores: '3-5' , node:'#ANY'}");
+    assertEquals(Operand.RANGE_EQUAL, clause.globalTag.op);
+    assertEquals(3.0d, ((RangeVal) clause.globalTag.val).min.doubleValue(), 0.001);
+    assertEquals(5.0d, ((RangeVal) clause.globalTag.val).max.doubleValue(), 0.001);
+
+    clause = Clause.create("{cores: 1.66 , node:'#ANY'}");
+    assertEquals(Operand.RANGE_EQUAL, clause.globalTag.op);
+    assertEquals(1.0d, ((RangeVal) clause.globalTag.val).min.doubleValue(), 0.001);
+    assertEquals(2.0d, ((RangeVal) clause.globalTag.val).max.doubleValue(), 0.001);
+    assertEquals(1.66d, ((RangeVal) clause.globalTag.val).actual.doubleValue(), 0.001);
+
+    expectThrows(IllegalArgumentException.class,
+        () -> Clause.create("{cores:5, sysprop.zone : west}"));
+
+    clause = Clause.create("{cores: '14%' , node:'#ANY'}");
+    assertEquals(ComputedType.PERCENT, clause.getGlobalTag().computedType);
+
+    clause = Clause.create("{cores: '14%' , node:[node1, node2, node3]}");
+    assertEquals(Operand.IN, clause.getTag().op);
   }
 
 
@@ -1018,7 +1041,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
       } else if (violation.node.equals("node5")) {
         assertEquals(-1, violation.replicaCountDelta.doubleValue(), 0.01);
 
-      } else{
+      } else {
         fail();
       }
     }
@@ -1026,7 +1049,6 @@ public class TestPolicy extends SolrTestCaseJ4 {
 //    assertEquals("node1", violation.node);
 
 
-
   }
 
   private static void expectError(String name, Object val, String msg) {
@@ -2358,10 +2380,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
     List<Violation> violations = session.getViolations();
     assertEquals(2, violations.size());
     for (Violation violation : violations) {
-      if(violation.node.equals("10.0.0.6:8983_solr")){
+      if (violation.node.equals("10.0.0.6:8983_solr")) {
         assertEquals(1.0d, violation.replicaCountDelta, 0.01);
         assertEquals(1.53d, ((RangeVal) violation.getClause().getReplica().val).actual);
-      } else if(violation.node.equals("10.0.0.6:7574_solr")){
+      } else if (violation.node.equals("10.0.0.6:7574_solr")) {
         assertEquals(-1.0d, violation.replicaCountDelta, 0.01);
       }
 
@@ -2486,7 +2508,6 @@ public class TestPolicy extends SolrTestCaseJ4 {
   }
 
 
-
   public void testFreeDiskSuggestions() {
     String dataproviderdata = "{" +
         "  liveNodes:[node1,node2]," +
@@ -3082,9 +3103,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
       suggester = createSuggester(cloudManager, jsonObj, suggester);
     }
 
-    assertEquals("count = "+count ,0,count);
+    assertEquals("count = " + count, 0, count);
   }
-public void testUtilizeNodeFailure2() throws Exception {
+
+  public void testUtilizeNodeFailure2() throws Exception {
     String state = "{  'liveNodes':[" +
         "  '127.0.0.1:51075_solr'," +
         "  '127.0.0.1:51076_solr'," +
@@ -3166,7 +3188,7 @@ public void testUtilizeNodeFailure2() throws Exception {
       suggester = createSuggester(cloudManager, jsonObj, suggester);
     }
 
-    assertEquals("count = "+count ,1,count);
+    assertEquals("count = " + count, 1, count);
   }
 
   //SOLR-12358
@@ -3476,6 +3498,7 @@ public void testUtilizeNodeFailure2() throws Exception {
 
 
   }
+
   public void testViolationOutput() throws IOException {
     String autoScalingjson = "{" +
         "  'cluster-preferences': [" +
@@ -3534,7 +3557,7 @@ public void testUtilizeNodeFailure2() throws Exception {
     new SolrJSONWriter(writer)
         .writeObj(val)
         .close();
-    JSONWriter.write (writer, true, JsonTextWriter.JSON_NL_MAP, val);
+    JSONWriter.write(writer, true, JsonTextWriter.JSON_NL_MAP, val);
 
     Object root = Utils.fromJSONString(writer.toString());
     assertEquals(2l,
@@ -3547,7 +3570,7 @@ public void testUtilizeNodeFailure2() throws Exception {
   }
 
 
-  public void testFreediskPercentage(){
+  public void testFreediskPercentage() {
     String dataproviderdata = "{" +
         "  'liveNodes': [" +
         "    'node1:8983'," +