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 2017/06/02 13:57:31 UTC

lucene-solr:feature/autoscaling_solr7: SOLR-9735: validation of all input values to either string or long

Repository: lucene-solr
Updated Branches:
  refs/heads/feature/autoscaling_solr7 e2ae9992c -> 4e691f9e4


SOLR-9735: validation of all input values to either string or long


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

Branch: refs/heads/feature/autoscaling_solr7
Commit: 4e691f9e4ae4269d4fbf8d947cc82bb7123085e7
Parents: e2ae999
Author: Noble Paul <no...@apache.org>
Authored: Fri Jun 2 23:27:22 2017 +0930
Committer: Noble Paul <no...@apache.org>
Committed: Fri Jun 2 23:27:22 2017 +0930

----------------------------------------------------------------------
 .../autoscaling/AutoScalingHandlerTest.java     |  2 +-
 .../apache/solr/cloud/autoscaling/Clause.java   | 70 +++++++++++++-------
 .../apache/solr/cloud/autoscaling/Operand.java  | 52 ++-------------
 .../org/apache/solr/cloud/autoscaling/Row.java  |  2 +-
 .../solr/cloud/autoscaling/TestPolicy.java      | 42 ++++++------
 5 files changed, 77 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e691f9e/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
----------------------------------------------------------------------
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 44cbd67..8b0401b 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
@@ -259,7 +259,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
       assertEquals(5, node.size());
       assertNotNull(sortedNodeNames[i] = (String) node.get("node"));
       assertNotNull(node.get("cores"));
-      assertEquals(0, node.get("cores"));
+      assertEquals("0", String.valueOf(node.get("cores")));
       assertNotNull(node.get("freedisk"));
       assertNotNull(node.get("sysLoadAvg"));
       assertNotNull(node.get("heapUsage"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e691f9e/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Clause.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Clause.java b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Clause.java
index bf0ebfb..9b88d78 100644
--- a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Clause.java
+++ b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Clause.java
@@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.solr.cloud.autoscaling.Policy.ReplicaInfo;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.util.RetryUtil;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 
@@ -74,16 +75,8 @@ public class Clause implements MapWriter, Comparable<Clause> {
       if(m.get(REPLICA) == null){
         throw new RuntimeException(StrUtils.formatString("'replica' is required" + Utils.toJSONString(m)));
       }
-      Condition replica = parse(REPLICA, m);
-      try {
-        int replicaCount = Integer.parseInt(String.valueOf(replica.val));
-        if(replicaCount<0){
-          throw new RuntimeException("replica value sould be non null "+ Utils.toJSONString(m));
-        }
-        this.replica = new Condition(replica.name, replicaCount, replica.op);
-      } catch (NumberFormatException e) {
-        throw new RuntimeException("Only an integer value is supported for replica " + Utils.toJSONString(m));
-      }
+      this.replica = parse(REPLICA, m);
+      if (replica.op == WILDCARD) throw new RuntimeException("replica val cannot be null" + Utils.toJSONString(m));
       m.forEach((s, o) -> parseCondition(s, o));
     }
     if (tag == null)
@@ -117,7 +110,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
       if (this.isPerCollectiontag() && that.isPerCollectiontag()) {
         v = Integer.compare(this.replica.op.priority, that.replica.op.priority);
         if (v == 0) {
-          v = Integer.compare((Integer) this.replica.val, (Integer) that.replica.val);
+          v = Long.compare((Long) this.replica.val, (Long) that.replica.val);
           v = this.replica.op == LESS_THAN ? v : v * -1;
         }
         return v;
@@ -154,7 +147,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
     }
 
     boolean isPass(Object inputVal) {
-      return op.match(val, inputVal) == PASS;
+      return op.match(val, validate(name, inputVal, false)) == PASS;
     }
 
     boolean isPass(Row row) {
@@ -180,8 +173,28 @@ public class Clause implements MapWriter, Comparable<Clause> {
     Object val = m.get(s);
     try {
       String conditionName = s.trim();
-      String value = val == null ? null : String.valueOf(val).trim();
       Operand operand = null;
+      if (val == null) {
+        operand = WILDCARD;
+        expectedVal = Policy.ANY;
+      } else if (val instanceof String) {
+        String strVal = ((String) val).trim();
+        if (Policy.ANY.equals(strVal) || Policy.EACH.equals(strVal)) operand = WILDCARD;
+        else if (strVal.startsWith(NOT_EQUAL.operand)) operand = NOT_EQUAL;
+        else if (strVal.startsWith(GREATER_THAN.operand)) operand = GREATER_THAN;
+        else if (strVal.startsWith(LESS_THAN.operand)) operand = LESS_THAN;
+        else operand = EQUAL;
+        expectedVal = validate(s, strVal.substring(EQUAL == operand || WILDCARD == operand ? 0 : 1), true);
+      } else if (val instanceof Number) {
+        operand = EQUAL;
+        expectedVal = validate(s, val, true);
+      }
+/*
+
+
+        String value = val == null ? null : String.valueOf(val).trim();
+      if(WILDCARD)
+
       if ((expectedVal = WILDCARD.parse(value)) != null) {
         operand = WILDCARD;
       } else if ((expectedVal = NOT_EQUAL.parse(value)) != null) {
@@ -194,6 +207,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
         operand = EQUAL;
         expectedVal = EQUAL.parse(value);
       }
+*/
 
       return new Condition(conditionName, expectedVal, operand);
 
@@ -357,26 +371,34 @@ public class Clause implements MapWriter, Comparable<Clause> {
   }
 
 
-  public static Object validate(String name, Object val) {
+  /**
+   *
+   * @param name name of the condition
+   * @param val value of the condition
+   * @param isRuleVal is this provided in the rule
+   * @return actual validated value
+   */
+  public static Object validate(String name, Object val, boolean isRuleVal) {
     if (val == null) return null;
     ValidateInfo info = validatetypes.get(name);
-    if (info == null && name.startsWith(ImplicitSnitch.SYSPROP)) info = validatetypes.get(null);
+    if (info == null && name.startsWith(ImplicitSnitch.SYSPROP)) info = validatetypes.get("STRING");
     if (info == null) throw new RuntimeException("Unknown type :" + name);
     if (info.type == Long.class) {
       Long num = parseNumber(name, val);
-      if (info.min != null)
-        if (num < info.min) throw new RuntimeException(name + ": " + val + " must be greater than " + info.min);
-      if (info.max != null)
-        if (num > info.max) throw new RuntimeException(name + ": " + val + " must be less than " + info.max);
+      if (isRuleVal) {
+        if (info.min != null)
+          if (num < info.min) throw new RuntimeException(name + ": " + val + " must be greater than " + info.min);
+        if (info.max != null)
+          if (num > info.max) throw new RuntimeException(name + ": " + val + " must be less than " + info.max);
+      }
       return num;
     } else if (info.type == String.class) {
-      if (info.vals != null && !info.vals.contains(val))
+      if (isRuleVal && info.vals != null && !info.vals.contains(val))
         throw new RuntimeException(name + ": " + val + " must be one of " + StrUtils.join(info.vals, ','));
       return val;
     } else {
       throw new RuntimeException("Invalid type ");
     }
-
   }
 
   public static Long parseNumber(String name, Object val) {
@@ -384,7 +406,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
     Number num = 0;
     if (val instanceof String) {
       try {
-        num = Long.parseLong((String) val);
+        num = Long.parseLong(((String) val).trim());
       } catch (NumberFormatException e) {
         try {
           num = Double.parseDouble((String) val);
@@ -411,7 +433,9 @@ public class Clause implements MapWriter, Comparable<Clause> {
     validatetypes.put(ImplicitSnitch.CORES, new ValidateInfo(Long.class, null, 0l, Long.MAX_VALUE));
     validatetypes.put(ImplicitSnitch.SYSLOADAVG, new ValidateInfo(Long.class, null, 0l, 100l));
     validatetypes.put(ImplicitSnitch.HEAPUSAGE, new ValidateInfo(Long.class, null, 0l, Long.MAX_VALUE));
-    validatetypes.put(null, new ValidateInfo(String.class, null, null, null));
+    validatetypes.put("NUMBER", new ValidateInfo(Long.class, null, 0l, Long.MAX_VALUE));//generic number validation
+    validatetypes.put("STRING", new ValidateInfo(String.class, null, null, null));//generic string validation
+    validatetypes.put("node", new ValidateInfo(String.class, null, null, null));
     for (String ip : ImplicitSnitch.IP_SNITCHES) validatetypes.put(ip, new ValidateInfo(Long.class, null, 0l, 255l));
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e691f9e/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Operand.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Operand.java b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Operand.java
index 5371c25..bf36ef6 100644
--- a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Operand.java
+++ b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Operand.java
@@ -34,11 +34,6 @@ public enum Operand {
       return testVal == null ? NOT_APPLICABLE : PASS;
     }
 
-    @Override
-    public Object parse(String val) {
-      if (val == null) return ANY;
-      return ANY.equals(val) || Policy.EACH.equals(val) ? val : null;
-    }
   },
   EQUAL("", 0) {
     @Override
@@ -60,15 +55,9 @@ public enum Operand {
   },
   GREATER_THAN(">", 1) {
     @Override
-    public Object parse(String val) {
-      return checkNumeric(super.parse(val));
-    }
-
-
-    @Override
     public TestStatus match(Object ruleVal, Object testVal) {
       if (testVal == null) return NOT_APPLICABLE;
-      return compareNum(ruleVal, testVal) == 1 ? PASS : FAIL;
+     return getLong(testVal) > getLong(ruleVal) ? PASS: FAIL ;
     }
 
     @Override
@@ -80,7 +69,7 @@ public enum Operand {
     @Override
     public TestStatus match(Object ruleVal, Object testVal) {
       if (testVal == null) return NOT_APPLICABLE;
-      return compareNum(ruleVal, testVal) == -1 ? PASS : FAIL;
+      return getLong(testVal) < getLong(ruleVal) ? PASS: FAIL ;
     }
 
     @Override
@@ -88,10 +77,6 @@ public enum Operand {
       return actual < expected ? 0 : (expected ) - actual;
     }
 
-    @Override
-    public Object parse(String val) {
-      return checkNumeric(super.parse(val));
-    }
   };
   public final String operand;
   final int priority;
@@ -105,38 +90,15 @@ public enum Operand {
     return operand + expectedVal.toString();
   }
 
-  Integer checkNumeric(Object val) {
-    if (val == null) return null;
-    try {
-      return Integer.parseInt(val.toString());
-    } catch (NumberFormatException e) {
-      throw new RuntimeException("for operand " + operand + " the value must be numeric");
-    }
-  }
-
-  public Object parse(String val) {
-    if (operand.isEmpty()) return val;
-    return val.startsWith(operand) ? val.substring(1) : null;
-  }
-
   public TestStatus match(Object ruleVal, Object testVal) {
-    return Objects.equals(String.valueOf(ruleVal), String.valueOf(testVal)) ? PASS : FAIL;
+    return Objects.equals(ruleVal, testVal) ? PASS : FAIL;
   }
 
+  Long getLong(Object o) {
+    if (o instanceof Long) return (Long) o;
+    if(o instanceof Number ) return ((Number) o).longValue();
+    return Long.parseLong(String.valueOf(o));
 
-  public int compareNum(Object n1Val, Object n2Val) {
-    Integer n1 = (Integer) parseObj(n1Val, Integer.class);
-    Integer n2 = (Integer) parseObj(n2Val, Integer.class);
-    return n1 > n2 ? -1 : Objects.equals(n1, n2) ? 0 : 1;
-  }
-
-  Object parseObj(Object o, Class typ) {
-    if (o == null) return o;
-    if (typ == String.class) return String.valueOf(o);
-    if (typ == Integer.class) {
-      return Integer.parseInt(String.valueOf(o));
-    }
-    return o;
   }
 
   public Integer delta(Object expected, Object actual) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e691f9e/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Row.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Row.java b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Row.java
index a2546d0..92b4bac 100644
--- a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Row.java
+++ b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Row.java
@@ -48,7 +48,7 @@ class Row implements MapWriter {
     Map<String, Object> vals = dataProvider.getNodeValues(node, params);
     for (int i = 0; i < params.size(); i++) {
       String s = params.get(i);
-      cells[i] = new Cell(i, s, vals.get(s));
+      cells[i] = new Cell(i, s, Clause.validate(s,vals.get(s), false));
       if (NODE.equals(s)) cells[i].val = node;
       if (cells[i].val == null) anyValueMissing = true;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4e691f9e/solr/solrj/src/test/org/apache/solr/cloud/autoscaling/TestPolicy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/cloud/autoscaling/TestPolicy.java b/solr/solrj/src/test/org/apache/solr/cloud/autoscaling/TestPolicy.java
index 08e1e1b..6bdd7cf 100644
--- a/solr/solrj/src/test/org/apache/solr/cloud/autoscaling/TestPolicy.java
+++ b/solr/solrj/src/test/org/apache/solr/cloud/autoscaling/TestPolicy.java
@@ -113,10 +113,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
   public void testValidate() {
     expectError("replica", -1, "must be greater than" );
     expectError("replica","hello", "not a valid number" );
-    assertEquals( 1l,   Clause.validate("replica", "1"));
-    assertEquals("c",   Clause.validate("collection", "c"));
-    assertEquals( "s",   Clause.validate("shard", "s"));
-    assertEquals( "overseer",   Clause.validate("nodeRole", "overseer"));
+    assertEquals( 1l,   Clause.validate("replica", "1", true));
+    assertEquals("c",   Clause.validate("collection", "c", true));
+    assertEquals( "s",   Clause.validate("shard", "s",true));
+    assertEquals( "overseer",   Clause.validate("nodeRole", "overseer",true));
 
     expectError("nodeRole", "wrong","must be one of");
 
@@ -125,8 +125,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
     expectError("sysLoadAvg", "-1","must be greater than");
     expectError("sysLoadAvg", -1,"must be greater than");
 
-    assertEquals(12l,Clause.validate("sysLoadAvg", "12.46"));
-    assertEquals(12l,Clause.validate("sysLoadAvg", 12.46d));
+    assertEquals(12l,Clause.validate("sysLoadAvg", "12.46",true));
+    assertEquals(12l,Clause.validate("sysLoadAvg", 12.46d,true));
 
 
     expectError("ip_1", "300","must be less than ");
@@ -134,12 +134,12 @@ public class TestPolicy extends SolrTestCaseJ4 {
     expectError("ip_1", "-1","must be greater than");
     expectError("ip_1", -1,"must be greater than");
 
-    assertEquals(1l,Clause.validate("ip_1", "1"));
+    assertEquals(1l,Clause.validate("ip_1", "1",true));
 
     expectError("heapUsage", "-1","must be greater than");
     expectError("heapUsage", -1,"must be greater than");
-    assertEquals(69l,Clause.validate("heapUsage", "69.9"));
-    assertEquals(69l,Clause.validate("heapUsage", 69.9d));
+    assertEquals(69l,Clause.validate("heapUsage", "69.9",true));
+    assertEquals(69l,Clause.validate("heapUsage", 69.9d,true));
 
     expectError("port", "70000","must be less than ");
     expectError("port", 70000,"must be less than ");
@@ -153,7 +153,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
   private static void expectError(String name, Object val, String msg){
     try {
-      Clause.validate(name, val);
+      Clause.validate(name, val,true);
       fail("expected exception containing "+msg);
     } catch (Exception e) {
       assertTrue("expected exception containing "+msg,e.getMessage().contains(msg));
@@ -202,7 +202,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "    'policy1': [" +
         "      { 'replica': '1', 'sysprop.fs': 'ssd', 'shard': '#EACH'}," +
         "      { 'replica': '<2', 'shard': '#ANY', 'node': '#ANY'}," +
-        "      { 'replica': '<2', 'shard': '#EACH', 'rack': 'rack1'}" +
+        "      { 'replica': '<2', 'shard': '#EACH', 'sysprop.rack': 'rack1'}" +
         "    ]" +
         "  }" +
         "}");
@@ -213,7 +213,7 @@ 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("rack"));
+    assertEquals("rack1", clauses.get(2).original.get("sysprop.rack"));
     assertEquals("overseer", clauses.get(1).original.get("nodeRole"));
   }
 
@@ -223,13 +223,13 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "      { 'nodeRole':'overseer', replica: 0,  'strict':false}," +
         "      { 'replica':'<1', 'node':'node3', 'shard':'#EACH'}," +
         "      { 'replica':'<2', 'node':'#ANY', 'shard':'#EACH'}," +
-        "      { 'replica':1, 'rack':'rack1'}]" +
+        "      { 'replica':1, 'sysprop.rack':'rack1'}]" +
         "  }";
     Policy p = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
     List<Clause> clauses = new ArrayList<>(p.getClusterPolicy());
     Collections.sort(clauses);
     assertEquals("nodeRole", clauses.get(1).tag.name);
-    assertEquals("rack", clauses.get(0).tag.name);
+    assertEquals("sysprop.rack", clauses.get(0).tag.name);
   }
 
   public void testRules() throws IOException {
@@ -467,7 +467,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "{replica:0, 'nodeRole':'overseer','strict':false}," +
         "{'replica':'<1','node':'node3'}," +
         "{'replica':'<2','node':'#ANY','shard':'#EACH'}," +
-        "{'replica':'<3','shard':'#EACH','rack':'#ANY'}" +
+        "{'replica':'<3','shard':'#EACH','sysprop.rack':'#ANY'}" +
         "]" +
         "}" +
         "}";
@@ -476,7 +476,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4}," +
         "node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3}," +
         "node3:{cores:7, freedisk: 262, heapUsage:7834, rack: rack2}," +
-        "node4:{cores:8, freedisk: 375, heapUsage:16900, nodeRole:overseer, rack: rack1}" +
+        "node4:{cores:8, freedisk: 375, heapUsage:16900, nodeRole:overseer, sysprop.rack: rack1}" +
         "}");
     Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
     ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState);
@@ -593,17 +593,17 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "  'policies': {" +
         "    'policy1': [" +
         "      { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
-        "      { 'replica': '<2', 'shard': '#EACH', 'rack': 'rack1'}" +
+        "      { 'replica': '<2', 'shard': '#EACH', 'sysprop.rack': 'rack1'}" +
         "    ]" +
         "  }" +
         "}";
 
 
     Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
-        "node1:{cores:12, freedisk: 334, heap:10480, rack:rack3}," +
-        "node2:{cores:4, freedisk: 749, heap:6873, sysprop.fs : ssd, rack:rack1}," +
-        "node3:{cores:7, freedisk: 262, heap:7834, rack:rack4}," +
-        "node4:{cores:0, freedisk: 900, heap:16900, nodeRole:overseer, rack:rack2}" +
+        "node1:{cores:12, freedisk: 334, heap:10480, sysprop.rack:rack3}," +
+        "node2:{cores:4, freedisk: 749, heap:6873, sysprop.fs : ssd, sysprop.rack:rack1}," +
+        "node3:{cores:7, freedisk: 262, heap:7834, sysprop.rack:rack4}," +
+        "node4:{cores:0, freedisk: 900, heap:16900, nodeRole:overseer, sysprop.rack:rack2}" +
         "}");
 
     ClusterDataProvider dataProvider = new ClusterDataProvider() {