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/03/28 13:18:34 UTC

lucene-solr:feature/autoscaling: SOLR-10278: renamed RuleSorted to Policy

Repository: lucene-solr
Updated Branches:
  refs/heads/feature/autoscaling e3a46732b -> 83541fa96


SOLR-10278: renamed RuleSorted to Policy


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

Branch: refs/heads/feature/autoscaling
Commit: 83541fa96860adf8045eb208eefc789032ea8bf3
Parents: e3a4673
Author: Noble Paul <no...@apache.org>
Authored: Tue Mar 28 23:48:25 2017 +1030
Committer: Noble Paul <no...@apache.org>
Committed: Tue Mar 28 23:48:25 2017 +1030

----------------------------------------------------------------------
 .../apache/solr/recipe/AddReplicaSuggester.java |   4 +-
 .../src/java/org/apache/solr/recipe/Clause.java |   6 +-
 .../solr/recipe/MoveReplicaSuggester.java       |   6 +-
 .../java/org/apache/solr/recipe/Operand.java    |   4 +-
 .../src/java/org/apache/solr/recipe/Policy.java | 251 +++++++++++++++++++
 .../java/org/apache/solr/recipe/Preference.java |   8 +-
 .../src/java/org/apache/solr/recipe/Row.java    |  17 +-
 .../java/org/apache/solr/recipe/RuleSorter.java | 251 -------------------
 .../test/org/apache/solr/recipe/TestPolicy.java | 185 ++++++++++++++
 .../org/apache/solr/recipe/TestRuleSorter.java  | 185 --------------
 10 files changed, 459 insertions(+), 458 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/java/org/apache/solr/recipe/AddReplicaSuggester.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/AddReplicaSuggester.java b/solr/solrj/src/java/org/apache/solr/recipe/AddReplicaSuggester.java
index ace808d..31cfc19 100644
--- a/solr/solrj/src/java/org/apache/solr/recipe/AddReplicaSuggester.java
+++ b/solr/solrj/src/java/org/apache/solr/recipe/AddReplicaSuggester.java
@@ -20,8 +20,8 @@ package org.apache.solr.recipe;
 import java.util.Map;
 
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.recipe.RuleSorter.BaseSuggester;
-import org.apache.solr.recipe.RuleSorter.Session;
+import org.apache.solr.recipe.Policy.BaseSuggester;
+import org.apache.solr.recipe.Policy.Session;
 
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/java/org/apache/solr/recipe/Clause.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/Clause.java b/solr/solrj/src/java/org/apache/solr/recipe/Clause.java
index 3385836..203ab58 100644
--- a/solr/solrj/src/java/org/apache/solr/recipe/Clause.java
+++ b/solr/solrj/src/java/org/apache/solr/recipe/Clause.java
@@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.recipe.RuleSorter.ReplicaStat;
+import org.apache.solr.recipe.Policy.ReplicaStat;
 
 import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.params.CoreAdminParams.COLLECTION;
@@ -41,8 +41,8 @@ import static org.apache.solr.recipe.Operand.GREATER_THAN;
 import static org.apache.solr.recipe.Operand.LESS_THAN;
 import static org.apache.solr.recipe.Operand.NOT_EQUAL;
 import static org.apache.solr.recipe.Operand.WILDCARD;
-import static org.apache.solr.recipe.RuleSorter.ANY;
-import static org.apache.solr.recipe.RuleSorter.EACH;
+import static org.apache.solr.recipe.Policy.ANY;
+import static org.apache.solr.recipe.Policy.EACH;
 
 // a set of conditions in a policy
 public class Clause implements MapWriter {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/java/org/apache/solr/recipe/MoveReplicaSuggester.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/MoveReplicaSuggester.java b/solr/solrj/src/java/org/apache/solr/recipe/MoveReplicaSuggester.java
index 8927345..2466ede 100644
--- a/solr/solrj/src/java/org/apache/solr/recipe/MoveReplicaSuggester.java
+++ b/solr/solrj/src/java/org/apache/solr/recipe/MoveReplicaSuggester.java
@@ -21,8 +21,8 @@ import java.util.Map;
 
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.recipe.RuleSorter.BaseSuggester;
-import org.apache.solr.recipe.RuleSorter.Session;
+import org.apache.solr.recipe.Policy.BaseSuggester;
+import org.apache.solr.recipe.Policy.Session;
 
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
@@ -46,7 +46,7 @@ public class MoveReplicaSuggester  extends BaseSuggester{
     //iterate through elements and identify the least loaded
     for (int i = 0; i < matrix.size(); i++) {
       Row fromRow = matrix.get(i);
-      Pair<Row, RuleSorter.ReplicaStat> pair = fromRow.removeReplica(coll, shard);
+      Pair<Row, Policy.ReplicaStat> pair = fromRow.removeReplica(coll, shard);
       fromRow = pair.first();
       if(fromRow == null){
         //no such replica available

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/java/org/apache/solr/recipe/Operand.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/Operand.java b/solr/solrj/src/java/org/apache/solr/recipe/Operand.java
index 796f06f..0d457c7 100644
--- a/solr/solrj/src/java/org/apache/solr/recipe/Operand.java
+++ b/solr/solrj/src/java/org/apache/solr/recipe/Operand.java
@@ -21,7 +21,7 @@ import java.util.Objects;
 
 import org.apache.solr.recipe.Clause.TestStatus;
 import static org.apache.solr.recipe.Clause.TestStatus.*;
-import static org.apache.solr.recipe.RuleSorter.ANY;
+import static org.apache.solr.recipe.Policy.ANY;
 
 
 public enum Operand {
@@ -34,7 +34,7 @@ public enum Operand {
     @Override
     public Object parse(String val) {
       if(val == null) return ANY;
-      return ANY.equals(val) || RuleSorter.EACH.equals(val) ? val : null;
+      return ANY.equals(val) || Policy.EACH.equals(val) ? val : null;
     }
   },
   EQUAL(""),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/java/org/apache/solr/recipe/Policy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/Policy.java b/solr/solrj/src/java/org/apache/solr/recipe/Policy.java
new file mode 100644
index 0000000..222f756
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/recipe/Policy.java
@@ -0,0 +1,251 @@
+/*
+ * 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.recipe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.util.Utils;
+
+import static java.util.Collections.singletonList;
+import static java.util.stream.Collectors.toList;
+
+public class Policy {
+  public static final String EACH = "#EACH";
+  public static final String ANY = "#ANY";
+  List<Clause> clauses = new ArrayList<>();
+  List<Preference> preferences = new ArrayList<>();
+  List<String> params= new ArrayList<>();
+
+
+  public Policy(Map<String, Object> jsonMap) {
+    List<Map<String, Object>> l = getListOfMap("conditions", jsonMap);
+    clauses = l.stream().map(Clause::new).collect(toList());
+    l = getListOfMap("preferences", jsonMap);
+    preferences = l.stream().map(Preference::new).collect(toList());
+    for (int i = 0; i < preferences.size() - 1; i++) {
+      Preference preference = preferences.get(i);
+      preference.next = preferences.get(i + 1);
+    }
+
+    for (Clause c : clauses) params.add(c.tag.name);
+    for (Preference preference : preferences) {
+      if (params.contains(preference.name.name())) {
+        throw new RuntimeException(preference.name + " is repeated");
+      }
+      params.add(preference.name.toString());
+      preference.idx = params.size() - 1;
+    }
+  }
+
+
+  public class Session implements MapWriter {
+    final List<String> nodes;
+    final NodeValueProvider snitch;
+    final List<Row> matrix;
+    Set<String> collections = new HashSet<>();
+
+    Session(List<String> nodes, NodeValueProvider snitch) {
+      this.nodes = nodes;
+      this.snitch = snitch;
+      matrix = new ArrayList<>(nodes.size());
+      for (String node : nodes) matrix.add(new Row(node, params, snitch));
+      for (Row row : matrix) row.replicaInfo.forEach((s, e) -> collections.add(s));
+    }
+
+    List<Row> getMatrixCopy() {
+      return matrix.stream()
+          .map(Row::copy)
+          .collect(Collectors.toList());
+    }
+
+    Policy getRuleSorter() {
+      return Policy.this;
+
+    }
+
+    /**Apply the preferences and conditions
+     */
+    public void applyRules() {
+      if (!preferences.isEmpty()) {
+        //this is to set the approximate value according to the precision
+        ArrayList<Row> tmpMatrix = new ArrayList<>(matrix);
+        for (Preference p : preferences) {
+          Collections.sort(tmpMatrix, (r1, r2) -> p.compare(r1, r2, false));
+          p.setApproxVal(tmpMatrix);
+        }
+        //approximate values are set now. Let's do recursive sorting
+        Collections.sort(matrix, (r1, r2) -> preferences.get(0).compare(r1, r2, true));
+      }
+
+      if (!clauses.isEmpty()) {
+        for (Clause clause : clauses) {
+          for (Row row : matrix) {
+            clause.test(row);
+          }
+        }
+      }
+
+    }
+
+    public Map<String, List<Clause>> getViolations() {
+      return matrix.stream()
+          .filter(row -> !row.violations.isEmpty())
+          .collect(Collectors.toMap(r -> r.node, r -> r.violations));
+    }
+
+    public Map suggest(CollectionAction action, String collection, String shard) {
+      Suggester op = ops.get(action);
+      if (op == null) throw new UnsupportedOperationException(action.toString() + "is not supported");
+      return op.suggest(collection, shard, this);
+    }
+
+    @Override
+    public void writeMap(EntryWriter ew) throws IOException {
+      for (int i = 0; i < matrix.size(); i++) {
+        Row row = matrix.get(i);
+        ew.put(row.node, row);
+      }
+    }
+
+    @Override
+    public String toString() {
+      return Utils.toJSONString(toMap(new LinkedHashMap<>()));
+    }
+    public List<Row> getSorted(){
+      return Collections.unmodifiableList(matrix);
+    }
+  }
+
+
+  public Session createSession(List<String> nodes, NodeValueProvider snitch) {
+    return new Session(nodes, snitch);
+  }
+
+
+  static List<Map<String, Object>> getListOfMap(String key, Map<String, Object> jsonMap) {
+    Object o = jsonMap.get(key);
+    if (o != null) {
+      if (!(o instanceof List)) o = singletonList(o);
+      return (List) o;
+    } else {
+      return Collections.emptyList();
+    }
+  }
+
+
+  enum SortParam {
+    freedisk, cores, heap, cpu;
+
+    static SortParam get(String m) {
+      for (SortParam p : values()) if (p.name().equals(m)) return p;
+      throw new RuntimeException("Sort must be on one of these " + Arrays.asList(values()));
+    }
+
+  }
+
+  enum Sort {
+    maximize(1), minimize(-1);
+    final int sortval;
+
+    Sort(int i) {
+      sortval = i;
+    }
+
+    static Sort get(Map<String, Object> m) {
+      if (m.containsKey(maximize.name()) && m.containsKey(minimize.name())) {
+        throw new RuntimeException("Cannot have both 'maximize' and 'minimize'");
+      }
+      if (m.containsKey(maximize.name())) return maximize;
+      if (m.containsKey(minimize.name())) return minimize;
+      throw new RuntimeException("must have either 'maximize' or 'minimize'");
+    }
+  }
+
+
+  static class ReplicaStat implements MapWriter {
+    final String name;
+    Map<String, Object> variables;
+
+    ReplicaStat(String name, Map<String, Object> vals) {
+      this.name = name;
+      this.variables = vals;
+    }
+
+    @Override
+    public void writeMap(EntryWriter ew) throws IOException {
+      ew.put(name, variables);
+    }
+  }
+
+
+  interface NodeValueProvider {
+    Map<String, Object> getValues(String node, Collection<String> keys);
+
+    /**
+     * Get the details of each replica in a node. It attempts to fetch as much details about
+     * the replica as mentioned in the keys list
+     * <p>
+     * the format is {collection:shard :[{replicadetails}]}
+     */
+    Map<String, Map<String, List<ReplicaStat>>> getReplicaCounts(String node, Collection<String> keys);
+  }
+
+  interface Suggester {
+    Map<String, Object> suggest(String coll, String shard, Session session);
+
+  }
+
+  static class BaseSuggester {
+    final String coll;
+    final String shard;
+    final Policy.Session session;
+    List<Row> matrix;
+
+    BaseSuggester(String coll, String shard, Policy.Session session) {
+      this.coll = coll;
+      this.shard = shard;
+      this.session = session;
+      matrix = session.getMatrixCopy();
+    }
+
+
+  }
+
+  private static final Map<CollectionAction, Suggester> ops = new HashMap<>();
+
+  static {
+    ops.put(CollectionAction.ADDREPLICA, (coll, shard, session) -> new AddReplicaSuggester(coll, shard, session).get());
+    ops.put(CollectionAction.MOVEREPLICA, (coll, shard, session) -> new MoveReplicaSuggester(coll, shard, session).get());
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/java/org/apache/solr/recipe/Preference.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/Preference.java b/solr/solrj/src/java/org/apache/solr/recipe/Preference.java
index 770f176..56f387a 100644
--- a/solr/solrj/src/java/org/apache/solr/recipe/Preference.java
+++ b/solr/solrj/src/java/org/apache/solr/recipe/Preference.java
@@ -21,15 +21,15 @@ import java.util.List;
 import java.util.Map;
 
 class Preference {
-  final RuleSorter.SortParam name;
+  final Policy.SortParam name;
   Integer precision;
-  final RuleSorter.Sort sort;
+  final Policy.Sort sort;
   Preference next;
   public int idx;
 
   Preference(Map<String, Object> m) {
-    sort = RuleSorter.Sort.get(m);
-    name = RuleSorter.SortParam.get(m.get(sort.name()).toString());
+    sort = Policy.Sort.get(m);
+    name = Policy.SortParam.get(m.get(sort.name()).toString());
     Object p = m.getOrDefault("precision", 0);
     precision = p instanceof Number ? ((Number) p).intValue() : Integer.parseInt(p.toString());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/java/org/apache/solr/recipe/Row.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/Row.java b/solr/solrj/src/java/org/apache/solr/recipe/Row.java
index 216a391..a98a3a1 100644
--- a/solr/solrj/src/java/org/apache/solr/recipe/Row.java
+++ b/solr/solrj/src/java/org/apache/solr/recipe/Row.java
@@ -28,6 +28,7 @@ import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.recipe.Policy.ReplicaStat;
 
 import static org.apache.solr.common.params.CoreAdminParams.NODE;
 
@@ -35,11 +36,11 @@ import static org.apache.solr.common.params.CoreAdminParams.NODE;
 class Row implements MapWriter {
   public final String node;
   final Cell[] cells;
-  Map<String, Map<String, List<RuleSorter.ReplicaStat>>> replicaInfo;
+  Map<String, Map<String, List<ReplicaStat>>> replicaInfo;
   List<Clause> violations = new ArrayList<>();
   boolean anyValueMissing = false;
 
-  Row(String node, List<String> params, RuleSorter.NodeValueProvider snitch) {
+  Row(String node, List<String> params, Policy.NodeValueProvider snitch) {
     replicaInfo = snitch.getReplicaCounts(node, params);
     if (replicaInfo == null) replicaInfo = Collections.emptyMap();
     this.node = node;
@@ -53,7 +54,7 @@ class Row implements MapWriter {
     }
   }
 
-  Row(String node, Cell[] cells, boolean anyValueMissing, Map<String, Map<String, List<RuleSorter.ReplicaStat>>> replicaInfo, List<Clause> violations) {
+  Row(String node, Cell[] cells, boolean anyValueMissing, Map<String, Map<String, List<ReplicaStat>>> replicaInfo, List<Clause> violations) {
     this.node = node;
     this.cells = new Cell[cells.length];
     for (int i = 0; i < this.cells.length; i++) {
@@ -89,20 +90,20 @@ class Row implements MapWriter {
 
   Row addReplica(String coll, String shard) {
     Row row = copy();
-    Map<String, List<RuleSorter.ReplicaStat>> c = row.replicaInfo.get(coll);
+    Map<String, List<ReplicaStat>> c = row.replicaInfo.get(coll);
     if (c == null) row.replicaInfo.put(coll, c = new HashMap<>());
-    List<RuleSorter.ReplicaStat> s = c.get(shard);
+    List<ReplicaStat> s = c.get(shard);
     if (s == null) c.put(shard, s = new ArrayList<>());
     return row;
 
 
   }
 
-  Pair<Row, RuleSorter.ReplicaStat> removeReplica(String coll, String shard) {
+  Pair<Row, ReplicaStat> removeReplica(String coll, String shard) {
     Row row = copy();
-    Map<String, List<RuleSorter.ReplicaStat>> c = row.replicaInfo.get(coll);
+    Map<String, List<ReplicaStat>> c = row.replicaInfo.get(coll);
     if(c == null) return null;
-    List<RuleSorter.ReplicaStat> s = c.get(shard);
+    List<ReplicaStat> s = c.get(shard);
     if (s == null || s.isEmpty()) return null;
     return new Pair(row,s.remove(0));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/java/org/apache/solr/recipe/RuleSorter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/RuleSorter.java b/solr/solrj/src/java/org/apache/solr/recipe/RuleSorter.java
deleted file mode 100644
index 72246f7..0000000
--- a/solr/solrj/src/java/org/apache/solr/recipe/RuleSorter.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/*
- * 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.recipe;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.solr.common.MapWriter;
-import org.apache.solr.common.params.CollectionParams.CollectionAction;
-import org.apache.solr.common.util.Utils;
-
-import static java.util.Collections.singletonList;
-import static java.util.stream.Collectors.toList;
-
-public class RuleSorter {
-  public static final String EACH = "#EACH";
-  public static final String ANY = "#ANY";
-  List<Clause> clauses = new ArrayList<>();
-  List<Preference> preferences = new ArrayList<>();
-  List<String> params= new ArrayList<>();
-
-
-  public RuleSorter(Map<String, Object> jsonMap) {
-    List<Map<String, Object>> l = getListOfMap("conditions", jsonMap);
-    clauses = l.stream().map(Clause::new).collect(toList());
-    l = getListOfMap("preferences", jsonMap);
-    preferences = l.stream().map(Preference::new).collect(toList());
-    for (int i = 0; i < preferences.size() - 1; i++) {
-      Preference preference = preferences.get(i);
-      preference.next = preferences.get(i + 1);
-    }
-
-    for (Clause c : clauses) params.add(c.tag.name);
-    for (Preference preference : preferences) {
-      if (params.contains(preference.name.name())) {
-        throw new RuntimeException(preference.name + " is repeated");
-      }
-      params.add(preference.name.toString());
-      preference.idx = params.size() - 1;
-    }
-  }
-
-
-  public class Session implements MapWriter {
-    final List<String> nodes;
-    final NodeValueProvider snitch;
-    final List<Row> matrix;
-    Set<String> collections = new HashSet<>();
-
-    Session(List<String> nodes, NodeValueProvider snitch) {
-      this.nodes = nodes;
-      this.snitch = snitch;
-      matrix = new ArrayList<>(nodes.size());
-      for (String node : nodes) matrix.add(new Row(node, params, snitch));
-      for (Row row : matrix) row.replicaInfo.forEach((s, e) -> collections.add(s));
-    }
-
-    List<Row> getMatrixCopy() {
-      return matrix.stream()
-          .map(Row::copy)
-          .collect(Collectors.toList());
-    }
-
-    RuleSorter getRuleSorter() {
-      return RuleSorter.this;
-
-    }
-
-    /**Apply the preferences and conditions
-     */
-    public void applyRules() {
-      if (!preferences.isEmpty()) {
-        //this is to set the approximate value according to the precision
-        ArrayList<Row> tmpMatrix = new ArrayList<>(matrix);
-        for (Preference p : preferences) {
-          Collections.sort(tmpMatrix, (r1, r2) -> p.compare(r1, r2, false));
-          p.setApproxVal(tmpMatrix);
-        }
-        //approximate values are set now. Let's do recursive sorting
-        Collections.sort(matrix, (r1, r2) -> preferences.get(0).compare(r1, r2, true));
-      }
-
-      if (!clauses.isEmpty()) {
-        for (Clause clause : clauses) {
-          for (Row row : matrix) {
-            clause.test(row);
-          }
-        }
-      }
-
-    }
-
-    public Map<String, List<Clause>> getViolations() {
-      return matrix.stream()
-          .filter(row -> !row.violations.isEmpty())
-          .collect(Collectors.toMap(r -> r.node, r -> r.violations));
-    }
-
-    public Map suggest(CollectionAction action, String collection, String shard) {
-      Suggester op = ops.get(action);
-      if (op == null) throw new UnsupportedOperationException(action.toString() + "is not supported");
-      return op.suggest(collection, shard, this);
-    }
-
-    @Override
-    public void writeMap(EntryWriter ew) throws IOException {
-      for (int i = 0; i < matrix.size(); i++) {
-        Row row = matrix.get(i);
-        ew.put(row.node, row);
-      }
-    }
-
-    @Override
-    public String toString() {
-      return Utils.toJSONString(toMap(new LinkedHashMap<>()));
-    }
-    public List<Row> getSorted(){
-      return Collections.unmodifiableList(matrix);
-    }
-  }
-
-
-  public Session createSession(List<String> nodes, NodeValueProvider snitch) {
-    return new Session(nodes, snitch);
-  }
-
-
-  static List<Map<String, Object>> getListOfMap(String key, Map<String, Object> jsonMap) {
-    Object o = jsonMap.get(key);
-    if (o != null) {
-      if (!(o instanceof List)) o = singletonList(o);
-      return (List) o;
-    } else {
-      return Collections.emptyList();
-    }
-  }
-
-
-  enum SortParam {
-    freedisk, cores, heap, cpu;
-
-    static SortParam get(String m) {
-      for (SortParam p : values()) if (p.name().equals(m)) return p;
-      throw new RuntimeException("Sort must be on one of these " + Arrays.asList(values()));
-    }
-
-  }
-
-  enum Sort {
-    maximize(1), minimize(-1);
-    final int sortval;
-
-    Sort(int i) {
-      sortval = i;
-    }
-
-    static Sort get(Map<String, Object> m) {
-      if (m.containsKey(maximize.name()) && m.containsKey(minimize.name())) {
-        throw new RuntimeException("Cannot have both 'maximize' and 'minimize'");
-      }
-      if (m.containsKey(maximize.name())) return maximize;
-      if (m.containsKey(minimize.name())) return minimize;
-      throw new RuntimeException("must have either 'maximize' or 'minimize'");
-    }
-  }
-
-
-  static class ReplicaStat implements MapWriter {
-    final String name;
-    Map<String, Object> variables;
-
-    ReplicaStat(String name, Map<String, Object> vals) {
-      this.name = name;
-      this.variables = vals;
-    }
-
-    @Override
-    public void writeMap(EntryWriter ew) throws IOException {
-      ew.put(name, variables);
-    }
-  }
-
-
-  interface NodeValueProvider {
-    Map<String, Object> getValues(String node, Collection<String> keys);
-
-    /**
-     * Get the details of each replica in a node. It attempts to fetch as much details about
-     * the replica as mentioned in the keys list
-     * <p>
-     * the format is {collection:shard :[{replicadetails}]}
-     */
-    Map<String, Map<String, List<ReplicaStat>>> getReplicaCounts(String node, Collection<String> keys);
-  }
-
-  interface Suggester {
-    Map<String, Object> suggest(String coll, String shard, Session session);
-
-  }
-
-  static class BaseSuggester {
-    final String coll;
-    final String shard;
-    final RuleSorter.Session session;
-    List<Row> matrix;
-
-    BaseSuggester(String coll, String shard, RuleSorter.Session session) {
-      this.coll = coll;
-      this.shard = shard;
-      this.session = session;
-      matrix = session.getMatrixCopy();
-    }
-
-
-  }
-
-  private static final Map<CollectionAction, Suggester> ops = new HashMap<>();
-
-  static {
-    ops.put(CollectionAction.ADDREPLICA, (coll, shard, session) -> new AddReplicaSuggester(coll, shard, session).get());
-    ops.put(CollectionAction.MOVEREPLICA, (coll, shard, session) -> new MoveReplicaSuggester(coll, shard, session).get());
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/test/org/apache/solr/recipe/TestPolicy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/recipe/TestPolicy.java b/solr/solrj/src/test/org/apache/solr/recipe/TestPolicy.java
new file mode 100644
index 0000000..c9ea011
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/recipe/TestPolicy.java
@@ -0,0 +1,185 @@
+/*
+ * 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.recipe;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.common.util.ValidatingJsonMap;
+
+public class TestPolicy extends SolrTestCaseJ4 {
+
+  public void testOperands() {
+    Clause c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:'<2', node:'#ANY'}"));
+    assertFalse(c.replica.isPass(3));
+    assertFalse(c.replica.isPass(2));
+    assertTrue(c.replica.isPass(1));
+
+    c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:'>2', node:'#ANY'}"));
+    assertTrue(c.replica.isPass(3));
+    assertFalse(c.replica.isPass(2));
+    assertFalse(c.replica.isPass(1));
+
+    c = new Clause((Map<String, Object>) Utils.fromJSONString("{nodeRole:'!overseer'}"));
+    assertTrue(c.tag.isPass("OVERSEER"));
+    assertFalse(c.tag.isPass("overseer"));
+
+
+  }
+  public void testRuleParsing() throws IOException {
+    String rules = "{" +
+        "conditions:[{nodeRole:'!overseer', strict:false},{replica:'<1',node:node3}," +
+        "{replica:'<2',node:'#ANY', shard:'#EACH'}]," +
+        " preferences:[" +
+        "{minimize:cores , precision:2}," +
+        "{maximize:freedisk, precision:50}, " +
+        "{minimize:heap, precision:1000}]}";
+
+
+    Map<String,Map> nodeValues = (Map<String, Map>) Utils.fromJSONString( "{" +
+        "node1:{cores:12, freedisk: 334, heap:10480}," +
+        "node2:{cores:4, freedisk: 749, heap:6873}," +
+        "node3:{cores:7, freedisk: 262, heap:7834}," +
+        "node4:{cores:8, freedisk: 375, heap:16900, nodeRole:overseer}" +
+        "}");
+    String clusterState = "{'gettingstarted':{" +
+        "    'router':{'name':'compositeId'}," +
+        "    'shards':{" +
+        "      'shard1':{" +
+        "        'range':'80000000-ffffffff'," +
+        "        'replicas':{" +
+        "          'r1':{" +
+        "            'core':r1," +
+        "            'base_url':'http://10.0.0.4:8983/solr'," +
+        "            'node_name':'node1'," +
+        "            'state':'active'," +
+        "            'leader':'true'}," +
+        "          'r2':{" +
+        "            'core':r2," +
+        "            'base_url':'http://10.0.0.4:7574/solr'," +
+        "            'node_name':'node2'," +
+        "            'state':'active'}}}," +
+        "      'shard2':{" +
+        "        'range':'0-7fffffff'," +
+        "        'replicas':{" +
+        "          'r3':{" +
+        "            'core':r3," +
+        "            'base_url':'http://10.0.0.4:8983/solr'," +
+        "            'node_name':'node1'," +
+        "            'state':'active'," +
+        "            'leader':'true'}," +
+        "          'r4':{" +
+        "            'core':r4," +
+        "            'base_url':'http://10.0.0.4:8987/solr'," +
+        "            'node_name':'node4'," +
+        "            'state':'active'}," +
+        "          'r6':{" +
+        "            'core':r6," +
+        "            'base_url':'http://10.0.0.4:8989/solr'," +
+        "            'node_name':'node3'," +
+        "            'state':'active'}," +
+        "          'r5':{" +
+        "            'core':r5," +
+        "            'base_url':'http://10.0.0.4:7574/solr'," +
+        "            'node_name':'node1'," +
+        "            'state':'active'}}}}}}";
+
+
+    ValidatingJsonMap m = ValidatingJsonMap
+        .getDeepCopy((Map) Utils.fromJSONString(clusterState), 6, true);
+
+
+    Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
+    Policy.Session session;
+    Policy.NodeValueProvider snitch = new Policy.NodeValueProvider() {
+      @Override
+      public Map<String,Object> getValues(String node, Collection<String> keys) {
+        Map<String,Object> result = new LinkedHashMap<>();
+        keys.stream().forEach(s -> result.put(s, nodeValues.get(node).get(s)));
+        return result;
+      }
+
+      @Override
+      public Map<String, Map<String, List<Policy.ReplicaStat>>> getReplicaCounts(String node, Collection<String> keys) {
+        Map<String, Map<String, List<Policy.ReplicaStat>>> result = new LinkedHashMap<>();
+
+        m.forEach((collName, o) -> {
+          ValidatingJsonMap coll = (ValidatingJsonMap) o;
+          coll.getMap("shards").forEach((shard, o1) -> {
+            ValidatingJsonMap sh = (ValidatingJsonMap) o1;
+            sh.getMap("replicas").forEach((replicaName, o2) -> {
+              ValidatingJsonMap r = (ValidatingJsonMap) o2;
+              String node_name = (String) r.get("node_name");
+              if (!node_name.equals(node)) return;
+              Map<String, List<Policy.ReplicaStat>> shardVsReplicaStats = result.get(collName);
+              if (shardVsReplicaStats == null) result.put(collName, shardVsReplicaStats = new HashMap<>());
+              List<Policy.ReplicaStat> replicaStats = shardVsReplicaStats.get(shard);
+              if (replicaStats == null) shardVsReplicaStats.put(shard, replicaStats = new ArrayList<>());
+              replicaStats.add(new Policy.ReplicaStat(replicaName, new HashMap<>()));
+            });
+          });
+        });
+
+        return result;
+      }
+
+
+    };
+
+    session = policy.createSession(Arrays.asList("node1", "node2", "node3", "node4"), snitch);
+
+    session.applyRules();
+    List<Row> l = session.getSorted();
+    assertEquals("node1",l.get(0).node);
+    assertEquals("node3",l.get(1).node);
+    assertEquals("node4",l.get(2).node);
+    assertEquals("node2",l.get(3).node);
+
+
+    System.out.printf(Utils.toJSONString(Utils.getDeepCopy(session.toMap(new LinkedHashMap<>()), 8)));
+    Map<String, List<Clause>> violations = session.getViolations();
+    System.out.println(Utils.getDeepCopy(violations, 6));
+    assertEquals(3, violations.size());
+    List<Clause> v = violations.get("node4");
+    assertNotNull(v);
+    assertEquals(v.get(0).tag.name, "nodeRole");
+    v = violations.get("node1");
+    assertNotNull(v);
+    assertEquals(v.get(0).replica.op, Operand.LESS_THAN);
+    assertEquals(v.get(0).replica.val, 2);
+    v = violations.get("node3");
+    assertNotNull(v);
+    assertEquals(v.get(0).replica.op, Operand.LESS_THAN);
+    assertEquals(v.get(0).replica.val, 1);
+    assertEquals(v.get(0).tag.val, "node3");
+
+
+
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/83541fa9/solr/solrj/src/test/org/apache/solr/recipe/TestRuleSorter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/recipe/TestRuleSorter.java b/solr/solrj/src/test/org/apache/solr/recipe/TestRuleSorter.java
deleted file mode 100644
index e69ad99..0000000
--- a/solr/solrj/src/test/org/apache/solr/recipe/TestRuleSorter.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * 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.recipe;
-
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.common.util.ValidatingJsonMap;
-
-public class TestRuleSorter extends SolrTestCaseJ4 {
-
-  public void testOperands() {
-    Clause c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:'<2', node:'#ANY'}"));
-    assertFalse(c.replica.isPass(3));
-    assertFalse(c.replica.isPass(2));
-    assertTrue(c.replica.isPass(1));
-
-    c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:'>2', node:'#ANY'}"));
-    assertTrue(c.replica.isPass(3));
-    assertFalse(c.replica.isPass(2));
-    assertFalse(c.replica.isPass(1));
-
-    c = new Clause((Map<String, Object>) Utils.fromJSONString("{nodeRole:'!overseer'}"));
-    assertTrue(c.tag.isPass("OVERSEER"));
-    assertFalse(c.tag.isPass("overseer"));
-
-
-  }
-  public void testRuleParsing() throws IOException {
-    String rules = "{" +
-        "conditions:[{nodeRole:'!overseer', strict:false},{replica:'<1',node:node3}," +
-        "{replica:'<2',node:'#ANY', shard:'#EACH'}]," +
-        " preferences:[" +
-        "{minimize:cores , precision:2}," +
-        "{maximize:freedisk, precision:50}, " +
-        "{minimize:heap, precision:1000}]}";
-
-
-    Map<String,Map> nodeValues = (Map<String, Map>) Utils.fromJSONString( "{" +
-        "node1:{cores:12, freedisk: 334, heap:10480}," +
-        "node2:{cores:4, freedisk: 749, heap:6873}," +
-        "node3:{cores:7, freedisk: 262, heap:7834}," +
-        "node4:{cores:8, freedisk: 375, heap:16900, nodeRole:overseer}" +
-        "}");
-    String clusterState = "{'gettingstarted':{" +
-        "    'router':{'name':'compositeId'}," +
-        "    'shards':{" +
-        "      'shard1':{" +
-        "        'range':'80000000-ffffffff'," +
-        "        'replicas':{" +
-        "          'r1':{" +
-        "            'core':r1," +
-        "            'base_url':'http://10.0.0.4:8983/solr'," +
-        "            'node_name':'node1'," +
-        "            'state':'active'," +
-        "            'leader':'true'}," +
-        "          'r2':{" +
-        "            'core':r2," +
-        "            'base_url':'http://10.0.0.4:7574/solr'," +
-        "            'node_name':'node2'," +
-        "            'state':'active'}}}," +
-        "      'shard2':{" +
-        "        'range':'0-7fffffff'," +
-        "        'replicas':{" +
-        "          'r3':{" +
-        "            'core':r3," +
-        "            'base_url':'http://10.0.0.4:8983/solr'," +
-        "            'node_name':'node1'," +
-        "            'state':'active'," +
-        "            'leader':'true'}," +
-        "          'r4':{" +
-        "            'core':r4," +
-        "            'base_url':'http://10.0.0.4:8987/solr'," +
-        "            'node_name':'node4'," +
-        "            'state':'active'}," +
-        "          'r6':{" +
-        "            'core':r6," +
-        "            'base_url':'http://10.0.0.4:8989/solr'," +
-        "            'node_name':'node3'," +
-        "            'state':'active'}," +
-        "          'r5':{" +
-        "            'core':r5," +
-        "            'base_url':'http://10.0.0.4:7574/solr'," +
-        "            'node_name':'node1'," +
-        "            'state':'active'}}}}}}";
-
-
-    ValidatingJsonMap m = ValidatingJsonMap
-        .getDeepCopy((Map) Utils.fromJSONString(clusterState), 6, true);
-
-
-    RuleSorter ruleSorter = new RuleSorter((Map<String, Object>) Utils.fromJSONString(rules));
-    RuleSorter.Session session;
-    RuleSorter.NodeValueProvider snitch = new RuleSorter.NodeValueProvider() {
-      @Override
-      public Map<String,Object> getValues(String node, Collection<String> keys) {
-        Map<String,Object> result = new LinkedHashMap<>();
-        keys.stream().forEach(s -> result.put(s, nodeValues.get(node).get(s)));
-        return result;
-      }
-
-      @Override
-      public Map<String, Map<String, List<RuleSorter.ReplicaStat>>> getReplicaCounts(String node, Collection<String> keys) {
-        Map<String, Map<String, List<RuleSorter.ReplicaStat>>> result = new LinkedHashMap<>();
-
-        m.forEach((collName, o) -> {
-          ValidatingJsonMap coll = (ValidatingJsonMap) o;
-          coll.getMap("shards").forEach((shard, o1) -> {
-            ValidatingJsonMap sh = (ValidatingJsonMap) o1;
-            sh.getMap("replicas").forEach((replicaName, o2) -> {
-              ValidatingJsonMap r = (ValidatingJsonMap) o2;
-              String node_name = (String) r.get("node_name");
-              if (!node_name.equals(node)) return;
-              Map<String, List<RuleSorter.ReplicaStat>> shardVsReplicaStats = result.get(collName);
-              if (shardVsReplicaStats == null) result.put(collName, shardVsReplicaStats = new HashMap<>());
-              List<RuleSorter.ReplicaStat> replicaStats = shardVsReplicaStats.get(shard);
-              if (replicaStats == null) shardVsReplicaStats.put(shard, replicaStats = new ArrayList<>());
-              replicaStats.add(new RuleSorter.ReplicaStat(replicaName, new HashMap<>()));
-            });
-          });
-        });
-
-        return result;
-      }
-
-
-    };
-
-    session = ruleSorter.createSession(Arrays.asList("node1", "node2", "node3", "node4"), snitch);
-
-    session.applyRules();
-    List<Row> l = session.getSorted();
-    assertEquals("node1",l.get(0).node);
-    assertEquals("node3",l.get(1).node);
-    assertEquals("node4",l.get(2).node);
-    assertEquals("node2",l.get(3).node);
-
-
-    System.out.printf(Utils.toJSONString(Utils.getDeepCopy(session.toMap(new LinkedHashMap<>()), 8)));
-    Map<String, List<Clause>> violations = session.getViolations();
-    System.out.println(Utils.getDeepCopy(violations, 6));
-    assertEquals(3, violations.size());
-    List<Clause> v = violations.get("node4");
-    assertNotNull(v);
-    assertEquals(v.get(0).tag.name, "nodeRole");
-    v = violations.get("node1");
-    assertNotNull(v);
-    assertEquals(v.get(0).replica.op, Operand.LESS_THAN);
-    assertEquals(v.get(0).replica.val, 2);
-    v = violations.get("node3");
-    assertNotNull(v);
-    assertEquals(v.get(0).replica.op, Operand.LESS_THAN);
-    assertEquals(v.get(0).replica.val, 1);
-    assertEquals(v.get(0).tag.val, "node3");
-
-
-
-  }
-
-
-}