You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/05/25 11:58:49 UTC

[1/5] lucene-solr:jira/solr-10515: SOLR-10738: Trigger has an init method which is called before schedule. Actions are init'ed in this method. Fixed NodeLostTriggerTest failures.

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-10515 53172a70c -> e4261d1cf


SOLR-10738: Trigger has an init method which is called before schedule. Actions are init'ed in this method. Fixed NodeLostTriggerTest failures.


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

Branch: refs/heads/jira/solr-10515
Commit: ce7367d54ec18940a7b778c2434ffc02bfe95a6e
Parents: fe21766
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Wed May 24 16:46:25 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Wed May 24 16:46:25 2017 +0530

----------------------------------------------------------------------
 .../org/apache/solr/cloud/autoscaling/AutoScaling.java  |  6 ++++++
 .../apache/solr/cloud/autoscaling/NodeAddedTrigger.java | 11 +++++++++++
 .../apache/solr/cloud/autoscaling/NodeLostTrigger.java  | 11 +++++++++++
 .../solr/cloud/autoscaling/ScheduledTriggers.java       |  5 +----
 .../solr/cloud/autoscaling/NodeLostTriggerTest.java     | 12 +++++++++++-
 5 files changed, 40 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce7367d5/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
index 5c992b0..08ceb27 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
@@ -115,6 +115,12 @@ public class AutoScaling {
     public boolean isClosed();
 
     public void restoreState(Trigger<E> old);
+
+    /**
+     * Called before a trigger is scheduled. Any heavy object creation or initialisation should
+     * be done in this method instead of the Trigger's constructor.
+     */
+    public void init();
   }
 
   public static class TriggerFactory implements Closeable {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce7367d5/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
index 1754a8b..93f8e84 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
@@ -82,6 +82,17 @@ public class NodeAddedTrigger implements AutoScaling.Trigger<NodeAddedTrigger.No
   }
 
   @Override
+  public void init() {
+    List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
+    if (o != null && !o.isEmpty()) {
+      for (int i = 0; i < o.size(); i++) {
+        Map<String, String> map = o.get(i);
+        actions.get(i).init(map);
+      }
+    }
+  }
+
+  @Override
   public void setListener(AutoScaling.TriggerListener<NodeAddedEvent> listener) {
     listenerRef.set(listener);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce7367d5/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
index 8bca93c..b755f25 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
@@ -83,6 +83,17 @@ public class NodeLostTrigger implements AutoScaling.Trigger<NodeLostTrigger.Node
   }
 
   @Override
+  public void init() {
+    List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
+    if (o != null && !o.isEmpty()) {
+      for (int i = 0; i < o.size(); i++) {
+        Map<String, String> map = o.get(i);
+        actions.get(i).init(map);
+      }
+    }
+  }
+
+  @Override
   public void setListener(AutoScaling.TriggerListener<NodeLostEvent> listener) {
     listenerRef.set(listener);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce7367d5/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
index 9c18ac2..37cb9c0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
@@ -143,10 +143,7 @@ public class ScheduledTriggers implements Closeable {
         return false;
       }
     });
-    List<TriggerAction> actions = newTrigger.getActions();
-    for (TriggerAction action : actions) {
-      action.init(newTrigger.getProperties());
-    }
+    newTrigger.init(); // mark as ready for scheduling
     scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(scheduledTrigger, 0, DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ce7367d5/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
index efa63d3..e570142 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
@@ -196,7 +196,17 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
     NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, container);
     trigger.setListener(noFirstRunListener);
     trigger.run();
-    newNode.stop();
+
+    // stop the newly created node
+    List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
+    for (int i = 0; i < jettySolrRunners.size(); i++) {
+      JettySolrRunner jettySolrRunner = jettySolrRunners.get(i);
+      if (newNode == jettySolrRunner) {
+        cluster.stopJettySolrRunner(i);
+        break;
+      }
+    }
+
     trigger.run(); // this run should detect the lost node
     trigger.close(); // close the old trigger
 


[2/5] lucene-solr:jira/solr-10515: SOLR-10738: Added tests for trigger.init() and TriggerAction lifecycle

Posted by ab...@apache.org.
SOLR-10738: Added tests for trigger.init() and TriggerAction lifecycle


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

Branch: refs/heads/jira/solr-10515
Commit: f9cf54963c7c159ee3389c079a0696920e7650fb
Parents: ce7367d
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu May 25 00:12:03 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu May 25 00:12:03 2017 +0530

----------------------------------------------------------------------
 .../cloud/autoscaling/NodeAddedTriggerTest.java | 62 ++++++++++++++
 .../cloud/autoscaling/NodeLostTriggerTest.java  | 63 ++++++++++++++
 .../autoscaling/TriggerIntegrationTest.java     | 90 +++++++++++++++-----
 3 files changed, 194 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9cf5496/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
index b0405cf..ec06b23 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.cloud.autoscaling;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -29,6 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -36,6 +38,9 @@ import org.junit.Test;
  * Test for {@link NodeAddedTrigger}
  */
 public class NodeAddedTriggerTest extends SolrCloudTestCase {
+  private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
 
   private AutoScaling.TriggerListener<NodeAddedTrigger.NodeAddedEvent> noFirstRunListener = event -> {
     fail("Did not expect the listener to fire on first run!");
@@ -49,6 +54,13 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
         .configure();
   }
 
+  @Before
+  public void beforeTest() throws Exception {
+    actionConstructorCalled = new AtomicBoolean(false);
+    actionInitCalled = new AtomicBoolean(false);
+    actionCloseCalled = new AtomicBoolean(false);
+  }
+
   @Test
   public void testTrigger() throws Exception {
     CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
@@ -123,6 +135,56 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     }
   }
 
+  public void testActionLifecycle() throws Exception {
+    CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
+    Map<String, Object> props = createTriggerProps(0);
+    List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
+    Map<String, String> action = new HashMap<>(2);
+    action.put("name", "testActionInit");
+    action.put("class", NodeAddedTriggerTest.AssertInitTriggerAction.class.getName());
+    actions.add(action);
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container)) {
+      assertEquals(true, actionConstructorCalled.get());
+      assertEquals(false, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+      trigger.init();
+      assertEquals(true, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+    }
+    assertEquals(true, actionCloseCalled.get());
+  }
+
+  public static class AssertInitTriggerAction implements TriggerAction  {
+    public AssertInitTriggerAction() {
+      actionConstructorCalled.set(true);
+    }
+
+    @Override
+    public String getName() {
+      return "";
+    }
+
+    @Override
+    public String getClassName() {
+      return getClass().getName();
+    }
+
+    @Override
+    public void process(AutoScaling.TriggerEvent event) {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+      actionCloseCalled.compareAndSet(false, true);
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      actionInitCalled.compareAndSet(false, true);
+    }
+  }
+
   @Test
   public void testListenerAcceptance() throws Exception {
     CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9cf5496/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
index e570142..9baae0f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.cloud.autoscaling;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -28,7 +29,9 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -36,6 +39,9 @@ import org.junit.Test;
  * Test for {@link NodeLostTrigger}
  */
 public class NodeLostTriggerTest extends SolrCloudTestCase {
+  private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
+  private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
 
   private AutoScaling.TriggerListener<NodeLostTrigger.NodeLostEvent> noFirstRunListener = event -> {
     fail("Did not expect the listener to fire on first run!");
@@ -49,6 +55,13 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
         .configure();
   }
 
+  @Before
+  public void beforeTest() throws Exception {
+    actionConstructorCalled = new AtomicBoolean(false);
+    actionInitCalled = new AtomicBoolean(false);
+    actionCloseCalled = new AtomicBoolean(false);
+  }
+
   @Test
   public void testTrigger() throws Exception {
     CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
@@ -136,6 +149,56 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
     }
   }
 
+  public void testActionLifecycle() throws Exception {
+    CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
+    Map<String, Object> props = createTriggerProps(0);
+    List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
+    Map<String, String> action = new HashMap<>(2);
+    action.put("name", "testActionInit");
+    action.put("class", AssertInitTriggerAction.class.getName());
+    actions.add(action);
+    try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, container)) {
+      assertEquals(true, actionConstructorCalled.get());
+      assertEquals(false, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+      trigger.init();
+      assertEquals(true, actionInitCalled.get());
+      assertEquals(false, actionCloseCalled.get());
+    }
+    assertEquals(true, actionCloseCalled.get());
+  }
+
+  public static class AssertInitTriggerAction implements TriggerAction  {
+    public AssertInitTriggerAction() {
+      actionConstructorCalled.set(true);
+    }
+
+    @Override
+    public String getName() {
+      return "";
+    }
+
+    @Override
+    public String getClassName() {
+      return getClass().getName();
+    }
+
+    @Override
+    public void process(AutoScaling.TriggerEvent event) {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+      actionCloseCalled.compareAndSet(false, true);
+    }
+
+    @Override
+    public void init(Map<String, String> args) {
+      actionInitCalled.compareAndSet(false, true);
+    }
+  }
+
   @Test
   public void testListenerAcceptance() throws Exception {
     CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9cf5496/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index db15f6c..7850f33 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@ -29,7 +29,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@@ -40,7 +39,6 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
-import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -58,7 +56,8 @@ import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_P
 public class TriggerIntegrationTest extends SolrCloudTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private static CountDownLatch actionCreated;
+  private static CountDownLatch actionConstructorCalled;
+  private static CountDownLatch actionInitCalled;
   private static CountDownLatch triggerFiredLatch;
   private static int waitForSeconds = 1;
   private static AtomicBoolean triggerFired;
@@ -76,7 +75,8 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   @Before
   public void setupTest() throws Exception {
     waitForSeconds = 1 + random().nextInt(3);
-    actionCreated = new CountDownLatch(1);
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
     triggerFiredLatch = new CountDownLatch(1);
     triggerFired = new AtomicBoolean(false);
     eventRef = new AtomicReference<>();
@@ -96,7 +96,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   @Test
   public void testTriggerThrottling() throws Exception  {
     // for this test we want to create two triggers so we must assert that the actions were created twice
-    TriggerIntegrationTest.actionCreated = new CountDownLatch(2);
+    TriggerIntegrationTest.actionInitCalled = new CountDownLatch(2);
     // similarly we want both triggers to fire
     triggerFiredLatch = new CountDownLatch(2);
 
@@ -129,7 +129,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
 
     // wait until the two instances of action are created
-    if (!actionCreated.await(3, TimeUnit.SECONDS))  {
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
       fail("Two TriggerAction instances should have been created by now");
     }
 
@@ -141,7 +141,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
 
     // reset shared state
     lastActionExecutedAt.set(0);
-    TriggerIntegrationTest.actionCreated = new CountDownLatch(2);
+    TriggerIntegrationTest.actionInitCalled = new CountDownLatch(2);
     triggerFiredLatch = new CountDownLatch(2);
 
     setTriggerCommand = "{" +
@@ -169,7 +169,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
 
     // wait until the two instances of action are created
-    if (!actionCreated.await(3, TimeUnit.SECONDS))  {
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
       fail("Two TriggerAction instances should have been created by now");
     }
 
@@ -231,7 +231,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   @Test
   public void testNodeLostTriggerRestoreState() throws Exception {
     // for this test we want to update the trigger so we must assert that the actions were created twice
-    TriggerIntegrationTest.actionCreated = new CountDownLatch(2);
+    TriggerIntegrationTest.actionInitCalled = new CountDownLatch(2);
 
     // start a new node
     JettySolrRunner newNode = cluster.startJettySolrRunner();
@@ -252,10 +252,10 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
 
     TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
-    while (actionCreated.getCount() == 0 && !timeOut.hasTimedOut()) {
+    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
       Thread.sleep(200);
     }
-    assertTrue("The action specified in node_lost_restore_trigger was not instantiated even after 2 seconds",actionCreated.getCount() > 0);
+    assertTrue("The action specified in node_lost_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
 
     List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
     int index = -1;
@@ -283,7 +283,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
 
     // wait until the second instance of action is created
-    if (!actionCreated.await(3, TimeUnit.SECONDS))  {
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
       fail("Two TriggerAction instances should have been created by now");
     }
 
@@ -299,7 +299,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   @Test
   public void testNodeAddedTriggerRestoreState() throws Exception {
     // for this test we want to update the trigger so we must assert that the actions were created twice
-    TriggerIntegrationTest.actionCreated = new CountDownLatch(2);
+    TriggerIntegrationTest.actionInitCalled = new CountDownLatch(2);
 
     CloudSolrClient solrClient = cluster.getSolrClient();
     waitForSeconds = 5;
@@ -316,10 +316,10 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
 
     TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
-    while (actionCreated.getCount() == 0 && !timeOut.hasTimedOut()) {
+    while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
       Thread.sleep(200);
     }
-    assertTrue("The action specified in node_added_restore_trigger was not instantiated even after 2 seconds",actionCreated.getCount() > 0);
+    assertTrue("The action specified in node_added_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
 
     // start a new node
     JettySolrRunner newNode = cluster.startJettySolrRunner();
@@ -341,7 +341,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(response.get("result").toString(), "success");
 
     // wait until the second instance of action is created
-    if (!actionCreated.await(3, TimeUnit.SECONDS))  {
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
       fail("Two TriggerAction instances should have been created by now");
     }
 
@@ -369,7 +369,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
-    if (!actionCreated.await(3, TimeUnit.SECONDS))  {
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
       fail("The TriggerAction should have been created by now");
     }
 
@@ -381,6 +381,30 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertNotNull(nodeAddedEvent);
     assertEquals("The node added trigger was fired but for a different node",
         newNode.getNodeName(), nodeAddedEvent.getNodeName());
+
+    // reset
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
+
+    // update the trigger with exactly the same data
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_added_trigger'," +
+        "'event' : 'nodeAdded'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = new AutoScalingHandlerTest.AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // this should be a no-op so the action should have been created but init should not be called
+    if (!actionConstructorCalled.await(3, TimeUnit.SECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    assertFalse(actionInitCalled.await(2, TimeUnit.SECONDS));
   }
 
   @Test
@@ -407,7 +431,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NamedList<Object> response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
 
-    if (!actionCreated.await(3, TimeUnit.SECONDS))  {
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
       fail("The TriggerAction should have been created by now");
     }
 
@@ -420,6 +444,30 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     assertNotNull(nodeLostEvent);
     assertEquals("The node lost trigger was fired but for a different node",
         lostNodeName, nodeLostEvent.getNodeName());
+
+    // reset
+    actionConstructorCalled = new CountDownLatch(1);
+    actionInitCalled = new CountDownLatch(1);
+
+    // update the trigger with exactly the same data
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '" + waitForSeconds + "s'," +
+        "'enabled' : true," +
+        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+        "}}";
+    req = new AutoScalingHandlerTest.AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
+    response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    // this should be a no-op so the action should have been created but init should not be called
+    if (!actionConstructorCalled.await(3, TimeUnit.SECONDS))  {
+      fail("The TriggerAction should have been created by now");
+    }
+
+    assertFalse(actionInitCalled.await(2, TimeUnit.SECONDS));
   }
 
   @Test
@@ -453,7 +501,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     SolrRequest req = new AutoScalingHandlerTest.AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
     response = solrClient.request(req);
     assertEquals(response.get("result").toString(), "success");
-    if (!actionCreated.await(3, TimeUnit.SECONDS))  {
+    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
       fail("The TriggerAction should have been created by now");
     }
 
@@ -473,7 +521,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   public static class TestTriggerAction implements TriggerAction {
 
     public TestTriggerAction() {
-
+      actionConstructorCalled.countDown();
     }
 
     @Override
@@ -507,7 +555,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     @Override
     public void init(Map<String, String> args) {
       log.info("TestTriggerAction init");
-      actionCreated.countDown();
+      actionInitCalled.countDown();
     }
   }
 }


[4/5] lucene-solr:jira/solr-10515: Merge remote-tracking branch 'origin/feature/autoscaling' into feature/autoscaling

Posted by ab...@apache.org.
Merge remote-tracking branch 'origin/feature/autoscaling' into feature/autoscaling


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

Branch: refs/heads/jira/solr-10515
Commit: b933b604070012700d5d2b89cb920214b0b6e355
Parents: 9ad811b f9cf549
Author: Noble Paul <no...@apache.org>
Authored: Thu May 25 17:55:10 2017 +0930
Committer: Noble Paul <no...@apache.org>
Committed: Thu May 25 17:55:10 2017 +0930

----------------------------------------------------------------------
 .../solr/cloud/autoscaling/AutoScaling.java     |  6 ++
 .../cloud/autoscaling/NodeAddedTrigger.java     | 11 +++
 .../solr/cloud/autoscaling/NodeLostTrigger.java | 11 +++
 .../cloud/autoscaling/ScheduledTriggers.java    |  5 +-
 .../cloud/autoscaling/NodeAddedTriggerTest.java | 62 ++++++++++++++
 .../cloud/autoscaling/NodeLostTriggerTest.java  | 75 +++++++++++++++-
 .../autoscaling/TriggerIntegrationTest.java     | 90 +++++++++++++++-----
 7 files changed, 234 insertions(+), 26 deletions(-)
----------------------------------------------------------------------



[5/5] lucene-solr:jira/solr-10515: Merge branch 'feature/autoscaling' into jira/solr-10515

Posted by ab...@apache.org.
Merge branch 'feature/autoscaling' into jira/solr-10515


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

Branch: refs/heads/jira/solr-10515
Commit: e4261d1cf71a9c3ceabd3ef682734636aac93438
Parents: 53172a7 b933b60
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Thu May 25 13:58:08 2017 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu May 25 13:58:08 2017 +0200

----------------------------------------------------------------------
 .../cloud/autoscaling/NodeAddedTriggerTest.java |  64 +++++++-
 .../cloud/autoscaling/NodeLostTriggerTest.java  |  62 ++++++++
 .../autoscaling/TriggerIntegrationTest.java     | 102 +++++++++----
 .../cloud/autoscaling/AddReplicaSuggester.java  |  21 ++-
 .../apache/solr/cloud/autoscaling/Clause.java   |  47 ++++--
 .../apache/solr/cloud/autoscaling/Operand.java  |  36 ++++-
 .../apache/solr/cloud/autoscaling/Policy.java   |  39 ++++-
 .../solr/cloud/autoscaling/TestPolicy.java      | 149 +++++++++++++------
 8 files changed, 416 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4261d1c/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
----------------------------------------------------------------------
diff --cc solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
index f1e1089,ec06b23..9730c5b
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
@@@ -29,7 -30,7 +30,8 @@@ import java.util.concurrent.atomic.Atom
  import org.apache.solr.client.solrj.embedded.JettySolrRunner;
  import org.apache.solr.cloud.SolrCloudTestCase;
  import org.apache.solr.core.CoreContainer;
 +import org.apache.solr.util.TimeSource;
+ import org.junit.Before;
  import org.junit.BeforeClass;
  import org.junit.Test;
  
@@@ -37,8 -38,11 +39,11 @@@
   * Test for {@link NodeAddedTrigger}
   */
  public class NodeAddedTriggerTest extends SolrCloudTestCase {
+   private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
+   private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
+   private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
  
 -  private AutoScaling.TriggerListener<NodeAddedTrigger.NodeAddedEvent> noFirstRunListener = event -> {
 +  private AutoScaling.TriggerListener noFirstRunListener = event -> {
      fail("Did not expect the listener to fire on first run!");
      return true;
    };
@@@ -85,9 -94,9 +97,9 @@@
          }
        } while (!fired.get());
  
 -      NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = eventRef.get();
 +      TriggerEvent nodeAddedEvent = eventRef.get();
        assertNotNull(nodeAddedEvent);
-       assertEquals("", newNode.getNodeName(), nodeAddedEvent.getProperty(NodeAddedTrigger.NodeAddedEvent.NODE_NAME));
 -      assertEquals("", newNode.getNodeName(), nodeAddedEvent.getNodeName());
++      assertEquals("", newNode.getNodeName(), nodeAddedEvent.getProperty(TriggerEvent.NODE_NAME));
      }
  
      // add a new node but remove it before the waitFor period expires
@@@ -126,6 -135,56 +138,56 @@@
      }
    }
  
+   public void testActionLifecycle() throws Exception {
+     CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
+     Map<String, Object> props = createTriggerProps(0);
+     List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
+     Map<String, String> action = new HashMap<>(2);
+     action.put("name", "testActionInit");
+     action.put("class", NodeAddedTriggerTest.AssertInitTriggerAction.class.getName());
+     actions.add(action);
+     try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container)) {
+       assertEquals(true, actionConstructorCalled.get());
+       assertEquals(false, actionInitCalled.get());
+       assertEquals(false, actionCloseCalled.get());
+       trigger.init();
+       assertEquals(true, actionInitCalled.get());
+       assertEquals(false, actionCloseCalled.get());
+     }
+     assertEquals(true, actionCloseCalled.get());
+   }
+ 
+   public static class AssertInitTriggerAction implements TriggerAction  {
+     public AssertInitTriggerAction() {
+       actionConstructorCalled.set(true);
+     }
+ 
+     @Override
+     public String getName() {
+       return "";
+     }
+ 
+     @Override
+     public String getClassName() {
+       return getClass().getName();
+     }
+ 
+     @Override
 -    public void process(AutoScaling.TriggerEvent event) {
++    public void process(TriggerEvent event) {
+ 
+     }
+ 
+     @Override
+     public void close() throws IOException {
+       actionCloseCalled.compareAndSet(false, true);
+     }
+ 
+     @Override
+     public void init(Map<String, String> args) {
+       actionInitCalled.compareAndSet(false, true);
+     }
+   }
+ 
    @Test
    public void testListenerAcceptance() throws Exception {
      CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4261d1c/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
----------------------------------------------------------------------
diff --cc solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
index d83c49f,9baae0f..6e35467
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
@@@ -28,8 -29,9 +29,9 @@@ import java.util.concurrent.atomic.Atom
  
  import org.apache.solr.client.solrj.embedded.JettySolrRunner;
  import org.apache.solr.cloud.SolrCloudTestCase;
 -import org.apache.solr.common.util.Utils;
  import org.apache.solr.core.CoreContainer;
 +import org.apache.solr.util.TimeSource;
+ import org.junit.Before;
  import org.junit.BeforeClass;
  import org.junit.Test;
  
@@@ -37,8 -39,11 +39,11 @@@
   * Test for {@link NodeLostTrigger}
   */
  public class NodeLostTriggerTest extends SolrCloudTestCase {
+   private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
+   private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
+   private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
  
 -  private AutoScaling.TriggerListener<NodeLostTrigger.NodeLostEvent> noFirstRunListener = event -> {
 +  private AutoScaling.TriggerListener noFirstRunListener = event -> {
      fail("Did not expect the listener to fire on first run!");
      return true;
    };
@@@ -140,6 -149,56 +152,56 @@@
      }
    }
  
+   public void testActionLifecycle() throws Exception {
+     CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
+     Map<String, Object> props = createTriggerProps(0);
+     List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
+     Map<String, String> action = new HashMap<>(2);
+     action.put("name", "testActionInit");
+     action.put("class", AssertInitTriggerAction.class.getName());
+     actions.add(action);
+     try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, container)) {
+       assertEquals(true, actionConstructorCalled.get());
+       assertEquals(false, actionInitCalled.get());
+       assertEquals(false, actionCloseCalled.get());
+       trigger.init();
+       assertEquals(true, actionInitCalled.get());
+       assertEquals(false, actionCloseCalled.get());
+     }
+     assertEquals(true, actionCloseCalled.get());
+   }
+ 
+   public static class AssertInitTriggerAction implements TriggerAction  {
+     public AssertInitTriggerAction() {
+       actionConstructorCalled.set(true);
+     }
+ 
+     @Override
+     public String getName() {
+       return "";
+     }
+ 
+     @Override
+     public String getClassName() {
+       return getClass().getName();
+     }
+ 
+     @Override
 -    public void process(AutoScaling.TriggerEvent event) {
++    public void process(TriggerEvent event) {
+ 
+     }
+ 
+     @Override
+     public void close() throws IOException {
+       actionCloseCalled.compareAndSet(false, true);
+     }
+ 
+     @Override
+     public void init(Map<String, String> args) {
+       actionInitCalled.compareAndSet(false, true);
+     }
+   }
+ 
    @Test
    public void testListenerAcceptance() throws Exception {
      CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4261d1c/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --cc solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index 768fb16,7850f33..1c189ae
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@@ -57,14 -56,12 +57,15 @@@ import static org.apache.solr.common.cl
  public class TriggerIntegrationTest extends SolrCloudTestCase {
    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
  
-   private static CountDownLatch actionCreated;
+   private static CountDownLatch actionConstructorCalled;
+   private static CountDownLatch actionInitCalled;
    private static CountDownLatch triggerFiredLatch;
    private static int waitForSeconds = 1;
 +  private static CountDownLatch actionStarted;
 +  private static CountDownLatch actionInterrupted;
 +  private static CountDownLatch actionCompleted;
    private static AtomicBoolean triggerFired;
 -  private static AtomicReference<AutoScaling.TriggerEvent> eventRef;
 +  private static AtomicReference<TriggerEvent> eventRef;
  
    private String path;
  
@@@ -78,35 -72,13 +79,32 @@@
          .configure();
    }
  
-   private static CountDownLatch getActionCreated() {
-     return actionCreated;
-   }
- 
 +  private static CountDownLatch getTriggerFiredLatch() {
 +    return triggerFiredLatch;
 +  }
 +
 +  private static CountDownLatch getActionStarted() {
 +    return actionStarted;
 +  }
 +
 +  private static CountDownLatch getActionInterrupted() {
 +    return actionInterrupted;
 +  }
 +
 +  private static CountDownLatch getActionCompleted() {
 +    return actionCompleted;
 +  }
 +
    @Before
    public void setupTest() throws Exception {
      waitForSeconds = 1 + random().nextInt(3);
-     actionCreated = new CountDownLatch(1);
+     actionConstructorCalled = new CountDownLatch(1);
+     actionInitCalled = new CountDownLatch(1);
      triggerFiredLatch = new CountDownLatch(1);
      triggerFired = new AtomicBoolean(false);
 +    actionStarted = new CountDownLatch(1);
 +    actionInterrupted = new CountDownLatch(1);
 +    actionCompleted = new CountDownLatch(1);
      eventRef = new AtomicReference<>();
      // clear any persisted auto scaling configuration
      Stat stat = zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
@@@ -124,7 -96,7 +122,7 @@@
    @Test
    public void testTriggerThrottling() throws Exception  {
      // for this test we want to create two triggers so we must assert that the actions were created twice
-     actionCreated = new CountDownLatch(2);
 -    TriggerIntegrationTest.actionInitCalled = new CountDownLatch(2);
++    actionInitCalled = new CountDownLatch(2);
      // similarly we want both triggers to fire
      triggerFiredLatch = new CountDownLatch(2);
  
@@@ -408,7 -380,31 +406,31 @@@
      NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = (NodeAddedTrigger.NodeAddedEvent) eventRef.get();
      assertNotNull(nodeAddedEvent);
      assertEquals("The node added trigger was fired but for a different node",
-         newNode.getNodeName(), nodeAddedEvent.getProperty(NodeAddedTrigger.NodeAddedEvent.NODE_NAME));
 -        newNode.getNodeName(), nodeAddedEvent.getNodeName());
++        newNode.getNodeName(), nodeAddedEvent.getProperty(TriggerEvent.NODE_NAME));
+ 
+     // reset
+     actionConstructorCalled = new CountDownLatch(1);
+     actionInitCalled = new CountDownLatch(1);
+ 
+     // update the trigger with exactly the same data
+     setTriggerCommand = "{" +
+         "'set-trigger' : {" +
+         "'name' : 'node_added_trigger'," +
+         "'event' : 'nodeAdded'," +
+         "'waitFor' : '" + waitForSeconds + "s'," +
+         "'enabled' : true," +
+         "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+         "}}";
+     req = new AutoScalingHandlerTest.AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
+     response = solrClient.request(req);
+     assertEquals(response.get("result").toString(), "success");
+ 
+     // this should be a no-op so the action should have been created but init should not be called
+     if (!actionConstructorCalled.await(3, TimeUnit.SECONDS))  {
+       fail("The TriggerAction should have been created by now");
+     }
+ 
+     assertFalse(actionInitCalled.await(2, TimeUnit.SECONDS));
    }
  
    @Test
@@@ -447,7 -443,31 +469,31 @@@
      NodeLostTrigger.NodeLostEvent nodeLostEvent = (NodeLostTrigger.NodeLostEvent) eventRef.get();
      assertNotNull(nodeLostEvent);
      assertEquals("The node lost trigger was fired but for a different node",
-         lostNodeName, nodeLostEvent.getProperty(NodeLostTrigger.NodeLostEvent.NODE_NAME));
 -        lostNodeName, nodeLostEvent.getNodeName());
++        lostNodeName, nodeLostEvent.getProperty(TriggerEvent.NODE_NAME));
+ 
+     // reset
+     actionConstructorCalled = new CountDownLatch(1);
+     actionInitCalled = new CountDownLatch(1);
+ 
+     // update the trigger with exactly the same data
+     setTriggerCommand = "{" +
+         "'set-trigger' : {" +
+         "'name' : 'node_lost_trigger'," +
+         "'event' : 'nodeLost'," +
+         "'waitFor' : '" + waitForSeconds + "s'," +
+         "'enabled' : true," +
+         "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
+         "}}";
+     req = new AutoScalingHandlerTest.AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
+     response = solrClient.request(req);
+     assertEquals(response.get("result").toString(), "success");
+ 
+     // this should be a no-op so the action should have been created but init should not be called
+     if (!actionConstructorCalled.await(3, TimeUnit.SECONDS))  {
+       fail("The TriggerAction should have been created by now");
+     }
+ 
+     assertFalse(actionInitCalled.await(2, TimeUnit.SECONDS));
    }
  
    @Test
@@@ -540,155 -555,7 +586,155 @@@
      @Override
      public void init(Map<String, String> args) {
        log.info("TestTriggerAction init");
-       actionCreated.countDown();
+       actionInitCalled.countDown();
      }
    }
 +
 +  public static class TestEventQueueAction implements TriggerAction {
 +
 +    public TestEventQueueAction() {
 +      log.info("TestEventQueueAction instantiated");
 +    }
 +
 +    @Override
 +    public String getName() {
 +      return this.getClass().getSimpleName();
 +    }
 +
 +    @Override
 +    public String getClassName() {
 +      return this.getClass().getName();
 +    }
 +
 +    @Override
 +    public void process(TriggerEvent event) {
 +      eventRef.set(event);
 +      getActionStarted().countDown();
 +      try {
 +        Thread.sleep(5000);
 +        triggerFired.compareAndSet(false, true);
 +        getActionCompleted().countDown();
 +      } catch (InterruptedException e) {
 +        getActionInterrupted().countDown();
 +        return;
 +      }
 +    }
 +
 +    @Override
 +    public void close() throws IOException {
 +
 +    }
 +
 +    @Override
 +    public void init(Map<String, String> args) {
 +      log.debug("TestTriggerAction init");
-       getActionCreated().countDown();
++      actionInitCalled.countDown();
 +    }
 +  }
 +
 +  @Test
 +  public void testEventQueue() throws Exception {
 +    CloudSolrClient solrClient = cluster.getSolrClient();
 +    String setTriggerCommand = "{" +
 +        "'set-trigger' : {" +
 +        "'name' : 'node_added_trigger1'," +
 +        "'event' : 'nodeAdded'," +
 +        "'waitFor' : '" + waitForSeconds + "s'," +
 +        "'enabled' : true," +
 +        "'actions' : [{'name':'test','class':'" + TestEventQueueAction.class.getName() + "'}]" +
 +        "}}";
 +    NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
 +    String overseerLeader = (String) overSeerStatus.get("leader");
 +    int overseerLeaderIndex = 0;
 +    for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
 +      JettySolrRunner jetty = cluster.getJettySolrRunner(i);
 +      if (jetty.getNodeName().equals(overseerLeader)) {
 +        overseerLeaderIndex = i;
 +        break;
 +      }
 +    }
 +    SolrRequest req = new AutoScalingHandlerTest.AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
 +    NamedList<Object> response = solrClient.request(req);
 +    assertEquals(response.get("result").toString(), "success");
 +
-     if (!actionCreated.await(3, TimeUnit.SECONDS))  {
++    if (!actionInitCalled.await(3, TimeUnit.SECONDS))  {
 +      fail("The TriggerAction should have been created by now");
 +    }
 +
 +    // add node to generate the event
 +    JettySolrRunner newNode = cluster.startJettySolrRunner();
 +    boolean await = actionStarted.await(60, TimeUnit.SECONDS);
 +    assertTrue("action did not start", await);
 +    // event should be there
 +    NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = (NodeAddedTrigger.NodeAddedEvent) eventRef.get();
 +    assertNotNull(nodeAddedEvent);
 +    // but action did not complete yet so the event is still enqueued
 +    assertFalse(triggerFired.get());
 +    actionStarted = new CountDownLatch(1);
 +    // kill overseer leader
 +    cluster.stopJettySolrRunner(overseerLeaderIndex);
 +    Thread.sleep(5000);
 +    await = actionInterrupted.await(3, TimeUnit.SECONDS);
 +    assertTrue("action wasn't interrupted", await);
 +    // new overseer leader should be elected and run triggers
 +    newNode = cluster.startJettySolrRunner();
 +    // it should fire again but not complete yet
 +    await = actionStarted.await(60, TimeUnit.SECONDS);
 +    TriggerEvent replayedEvent = eventRef.get();
 +    assertTrue(replayedEvent.getProperty(TriggerEventQueue.ENQUEUE_TIME) != null);
 +    assertTrue(replayedEvent.getProperty(TriggerEventQueue.DEQUEUE_TIME) != null);
 +    await = actionCompleted.await(10, TimeUnit.SECONDS);
 +    assertTrue(triggerFired.get());
 +  }
 +
 +  @Test
 +  public void testEventFromRestoredState() throws Exception {
 +    CloudSolrClient solrClient = cluster.getSolrClient();
 +    String setTriggerCommand = "{" +
 +        "'set-trigger' : {" +
 +        "'name' : 'node_added_trigger'," +
 +        "'event' : 'nodeAdded'," +
 +        "'waitFor' : '10s'," +
 +        "'enabled' : true," +
 +        "'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
 +        "}}";
 +    SolrRequest req = new AutoScalingHandlerTest.AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
 +    NamedList<Object> response = solrClient.request(req);
 +    assertEquals(response.get("result").toString(), "success");
 +
-     if (!actionCreated.await(10, TimeUnit.SECONDS))  {
++    if (!actionInitCalled.await(10, TimeUnit.SECONDS))  {
 +      fail("The TriggerAction should have been created by now");
 +    }
 +
 +    NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
 +    String overseerLeader = (String) overSeerStatus.get("leader");
 +    int overseerLeaderIndex = 0;
 +    for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
 +      JettySolrRunner jetty = cluster.getJettySolrRunner(i);
 +      if (jetty.getNodeName().equals(overseerLeader)) {
 +        overseerLeaderIndex = i;
 +        break;
 +      }
 +    }
 +
 +    JettySolrRunner newNode = cluster.startJettySolrRunner();
 +    boolean await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
 +    assertTrue("The trigger did not fire at all", await);
 +    assertTrue(triggerFired.get());
 +    // reset
 +    triggerFired.set(false);
 +    triggerFiredLatch = new CountDownLatch(1);
 +    NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = (NodeAddedTrigger.NodeAddedEvent) eventRef.get();
 +    assertNotNull(nodeAddedEvent);
 +    assertEquals("The node added trigger was fired but for a different node",
 +        newNode.getNodeName(), nodeAddedEvent.getProperty(NodeAddedTrigger.NodeAddedEvent.NODE_NAME));
 +    // add a second node - state of the trigger will change but it won't fire for waitFor sec.
 +    JettySolrRunner newNode2 = cluster.startJettySolrRunner();
 +    Thread.sleep(10000);
 +    // kill overseer leader
 +    cluster.stopJettySolrRunner(overseerLeaderIndex);
 +    await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
 +    assertTrue("The trigger did not fire at all", await);
 +    assertTrue(triggerFired.get());
 +  }
  }


[3/5] lucene-solr:jira/solr-10515: fixed and added test for greedy conditions

Posted by ab...@apache.org.
fixed and added test for greedy conditions


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

Branch: refs/heads/jira/solr-10515
Commit: 9ad811b36f3538fc8e1beb0c1b1c438a3ca4db84
Parents: fe21766
Author: Noble Paul <no...@apache.org>
Authored: Thu May 25 17:55:00 2017 +0930
Committer: Noble Paul <no...@apache.org>
Committed: Thu May 25 17:55:00 2017 +0930

----------------------------------------------------------------------
 .../cloud/autoscaling/AddReplicaSuggester.java  |  21 ++-
 .../apache/solr/cloud/autoscaling/Clause.java   |  47 ++++--
 .../apache/solr/cloud/autoscaling/Operand.java  |  36 ++++-
 .../apache/solr/cloud/autoscaling/Policy.java   |  39 ++++-
 .../solr/cloud/autoscaling/TestPolicy.java      | 149 +++++++++++++------
 5 files changed, 217 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ad811b3/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/AddReplicaSuggester.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/AddReplicaSuggester.java b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/AddReplicaSuggester.java
index d0e510d..ac17ac2 100644
--- a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/AddReplicaSuggester.java
+++ b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/AddReplicaSuggester.java
@@ -38,6 +38,8 @@ class AddReplicaSuggester extends Suggester {
       throw new RuntimeException("add-replica requires 'collection' and 'shard'");
     //iterate through elements and identify the least loaded
 
+    List<Clause.Violation> leastSeriousViolation = null;
+    Integer targetNodeIndex = null;
     for (int i = getMatrix().size() - 1; i >= 0; i--) {
       Row row = getMatrix().get(i);
       if (!isAllowed(row.node, Hint.TARGET_NODE)) continue;
@@ -45,14 +47,21 @@ class AddReplicaSuggester extends Suggester {
       tmpRow.violations.clear();
 
       List<Clause.Violation> errs = testChangedRow(strict, getModifiedMatrix(getMatrix(), tmpRow, i));
-
-      if (!containsNewErrors(errs)) {// there are no rule violations
-        getMatrix().set(i, getMatrix().get(i).addReplica(coll, shard));
-        return CollectionAdminRequest
-            .addReplicaToShard(coll, shard)
-            .setNode(row.node);
+      if(!containsNewErrors(errs)) {
+        if(isLessSerious(errs, leastSeriousViolation)){
+          leastSeriousViolation = errs;
+          targetNodeIndex = i;
+        }
       }
     }
+
+    if (targetNodeIndex != null) {// there are no rule violations
+      getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(coll, shard));
+      return CollectionAdminRequest
+          .addReplicaToShard(coll, shard)
+          .setNode(getMatrix().get(targetNodeIndex).node);
+    }
+
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ad811b3/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 d466ce9..0406bc2 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
@@ -71,7 +71,13 @@ public class Clause implements MapWriter, Comparable<Clause> {
     } else {
       collection = parse(COLLECTION, m);
       shard = parse(SHARD, m);
-      this.replica = parse(REPLICA, m);
+      Condition replica = parse(REPLICA, m);
+      try {
+        int replicaCount = Integer.parseInt(String.valueOf(replica.val));
+        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));
+      }
       m.forEach((s, o) -> parseCondition(s, o));
     }
     if (tag == null)
@@ -102,12 +108,17 @@ public class Clause implements MapWriter, Comparable<Clause> {
     try {
       int v = Integer.compare(this.tag.op.priority, that.tag.op.priority);
       if (v != 0) return v;
-      return this.isPerCollectiontag() && that.isPerCollectiontag() ?
-          Integer.compare(this.replica.op.priority, that.replica.op.priority) :
-          0;
+      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 = this.replica.op == LESS_THAN ? v : v * -1;
+        }
+        return v;
+      } else {
+        return 0;
+      }
     } catch (NullPointerException e) {
-      System.out.println("this: " + Utils.toJSONString(this));
-      System.out.println("thAt: " + Utils.toJSONString(that));
       throw e;
     }
   }
@@ -147,6 +158,10 @@ public class Clause implements MapWriter, Comparable<Clause> {
       }
       return false;
     }
+
+    public Integer delta(Object val) {
+      return op.delta(this.val, val);
+    }
   }
 
   static Condition parse(String s, Map m) {
@@ -179,13 +194,15 @@ public class Clause implements MapWriter, Comparable<Clause> {
   public class Violation implements MapWriter {
     final String shard, coll, node;
     final Object actualVal;
+    final Integer delta;//how far is the actual value from the expected value
     private final int hash;
 
 
-    private Violation(String coll, String shard, String node, Object actualVal) {
+    private Violation(String coll, String shard, String node, Object actualVal, Integer delta ) {
       this.shard = shard;
       this.coll = coll;
       this.node = node;
+      this.delta = delta;
       this.actualVal = actualVal;
       hash = ("" + coll + " " + shard + " " + node + " " + Utils.toJSONString(getClause().toMap(new HashMap<>()))).hashCode();
     }
@@ -215,8 +232,11 @@ public class Clause implements MapWriter, Comparable<Clause> {
       ew.putIfNotNull("collection", coll);
       ew.putIfNotNull("shard", shard);
       ew.putIfNotNull("node", node);
-      ew.putIfNotNull("violation", (MapWriter) ew1 -> ew1.put(getClause().isPerCollectiontag() ? "replica" : tag.name,
-          String.valueOf(actualVal)));
+      ew.putIfNotNull("violation", (MapWriter) ew1 -> {
+        ew1.put(getClause().isPerCollectiontag() ? "replica" : tag.name,
+            String.valueOf(actualVal));
+        ew1.putIfNotNull("delta", delta);
+      });
       ew.put("clause", getClause());
     }
   }
@@ -233,7 +253,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
           for (Map.Entry<String, AtomicInteger> counts : shardVsCount.getValue().entrySet()) {
             if (!replica.isPass(counts.getValue())) {
               errors.add(new Violation(e.getKey(), shardVsCount.getKey(),
-                  tag.name.equals("node") ? counts.getKey() : null, counts.getValue()));
+                  tag.name.equals("node") ? counts.getKey() : null, counts.getValue(), replica.delta(counts.getValue())));
             }
           }
         }
@@ -241,7 +261,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
     } else {
       for (Row r : allRows) {
         if (!tag.isPass(r)) {
-          errors.add(new Violation(null, null, r.node, r.getVal(tag.name) ));
+          errors.add(new Violation(null, null, r.node, r.getVal(tag.name) , tag.delta(r.getVal(tag.name))));
         }
       }
     }
@@ -264,11 +284,10 @@ public class Clause implements MapWriter, Comparable<Clause> {
           if (!shard.isPass(shardName)) break;
           collMap.putIfAbsent(shardName, new HashMap<>());
           Map<String, AtomicInteger> tagVsCount = collMap.get(shardName);
-          AtomicInteger count = null;
           Object tagVal = row.getVal(tag.name);
+          tagVsCount.putIfAbsent(tag.isPass(tagVal)? String.valueOf(tagVal) : "", new AtomicInteger());
           if (tag.isPass(tagVal)) {
-            tagVsCount.put(String.valueOf(tagVal), count = tagVsCount.getOrDefault(tagVal, new AtomicInteger()));
-            count.addAndGet(shards.getValue().size());
+            tagVsCount.get(tagVal).addAndGet(shards.getValue().size());
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ad811b3/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 6ff1ef5..5371c25 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
@@ -40,12 +40,23 @@ public enum Operand {
       return ANY.equals(val) || Policy.EACH.equals(val) ? val : null;
     }
   },
-  EQUAL("", 0),
+  EQUAL("", 0) {
+    @Override
+    public int _delta(int expected, int actual) {
+      return expected - actual;
+    }
+  },
   NOT_EQUAL("!", 2) {
     @Override
     public TestStatus match(Object ruleVal, Object testVal) {
       return super.match(ruleVal, testVal) == PASS ? FAIL : PASS;
     }
+
+    @Override
+    public int _delta(int expected, int actual) {
+      return expected - actual;
+    }
+
   },
   GREATER_THAN(">", 1) {
     @Override
@@ -60,6 +71,10 @@ public enum Operand {
       return compareNum(ruleVal, testVal) == 1 ? PASS : FAIL;
     }
 
+    @Override
+    protected int _delta(int expected, int actual) {
+      return actual > expected ? 0 : (expected + 1) - actual;
+    }
   },
   LESS_THAN("<", 2) {
     @Override
@@ -69,6 +84,11 @@ public enum Operand {
     }
 
     @Override
+    protected int _delta(int expected, int actual) {
+      return actual < expected ? 0 : (expected ) - actual;
+    }
+
+    @Override
     public Object parse(String val) {
       return checkNumeric(super.parse(val));
     }
@@ -118,4 +138,18 @@ public enum Operand {
     }
     return o;
   }
+
+  public Integer delta(Object expected, Object actual) {
+    try {
+      Integer expectedInt = Integer.parseInt(String.valueOf(expected));
+      Integer actualInt = Integer.parseInt(String.valueOf(actual));
+      return _delta(expectedInt, actualInt);
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  protected int _delta(int expected, int actual) {
+    return 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ad811b3/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Policy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Policy.java b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Policy.java
index 614bfa2..98a10f3 100644
--- a/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Policy.java
+++ b/solr/solrj/src/java/org/apache/solr/cloud/autoscaling/Policy.java
@@ -348,13 +348,20 @@ public class Policy implements MapWriter {
     public SolrRequest getOperation() {
       if (!isInitialized) {
         String coll = (String) hints.get(Hint.COLL);
-        if(coll != null){
-          // if this is not a known collection from the existing clusterstate,
-          // then add it
-          if(session.matrix.stream().noneMatch(row -> row.replicaInfo.containsKey(coll))){
-            session.matrix.get(0).replicaInfo.put(coll, new HashMap<>());
-            session.addClausesForCollection(session.dataProvider, coll);
-            Collections.sort(session.expandedClauses);
+        String shard = (String) hints.get(Hint.SHARD);
+        // if this is not a known collection from the existing clusterstate,
+        // then add it
+        if(session.matrix.stream().noneMatch(row -> row.replicaInfo.containsKey(coll))){
+          session.addClausesForCollection(session.dataProvider, coll);
+          Collections.sort(session.expandedClauses);
+        }
+        if(coll != null) {
+          for (Row row : session.matrix) {
+            if (!row.replicaInfo.containsKey(coll)) row.replicaInfo.put(coll, new HashMap<>());
+            if(shard != null){
+              Map<String, List<ReplicaInfo>> shardInfo = row.replicaInfo.get(coll);
+              if(!shardInfo.containsKey(shard)) shardInfo.put(shard, new ArrayList<>());
+            }
           }
         }
         session.applyRules();
@@ -373,7 +380,23 @@ public class Policy implements MapWriter {
       return session.matrix;
 
     }
-    boolean containsNewErrors(List<Clause.Violation> errs){
+
+    boolean isLessSerious(List<Violation> fresh, List<Violation> old) {
+      if (old == null || fresh.size() < old.size()) return true;
+      if(fresh.size() == old.size()){
+        for (int i = 0; i < old.size(); i++) {
+          if(fresh.get(i).equals(old.get(i))) {
+            if (fresh.get(i) != null &&
+                old.get(i).delta != null &&
+                Math.abs(fresh.get(i).delta) < Math.abs(old.get(i).delta))
+              return true;
+          }
+        }
+
+      }
+      return false;
+    }
+    boolean containsNewErrors(List<Violation> errs){
       for (Clause.Violation err : errs) {
         if(!originalViolations.contains(err)) return true;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ad811b3/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 36c18e0..962ae15 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
@@ -55,7 +55,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
     assertFalse(c.replica.isPass(2));
     assertFalse(c.replica.isPass(1));
 
-    c = new Clause((Map<String, Object>) Utils.fromJSONString("{nodeRole:'!overseer'}"));
+    c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:0, nodeRole:'!overseer'}"));
     assertTrue(c.tag.isPass("OVERSEER"));
     assertFalse(c.tag.isPass("overseer"));
   }
@@ -96,12 +96,12 @@ 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("rack"));
     assertEquals("overseer", clauses.get(1).original.get("nodeRole"));
   }
 
 
-  public void testConditionsSort(){
+  public void testConditionsSort() {
     String rules = "{" +
         "    'cluster-policy':[" +
         "      { 'nodeRole':'overseer', replica: 0,  'strict':false}," +
@@ -112,9 +112,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
     Policy p = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
     List<Clause> clauses = new ArrayList<>(p.getClusterPolicy());
     Collections.sort(clauses);
-    assertEquals("nodeRole", clauses.get(0).tag.name);
-    assertEquals("rack", clauses.get(1).tag.name);
+    assertEquals("nodeRole", clauses.get(1).tag.name);
+    assertEquals("rack", clauses.get(0).tag.name);
   }
+
   public static String clusterState = "{'gettingstarted':{" +
       "    'router':{'name':'compositeId'}," +
       "    'shards':{" +
@@ -188,8 +189,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
     List<Violation> violations = session.getViolations();
     assertEquals(3, violations.size());
-    assertTrue( violations.stream().anyMatch(violation -> "node3".equals(violation.getClause().tag.val)));
-    assertTrue( violations.stream().anyMatch(violation -> "nodeRole".equals(violation.getClause().tag.name)));
+    assertTrue(violations.stream().anyMatch(violation -> "node3".equals(violation.getClause().tag.val)));
+    assertTrue(violations.stream().anyMatch(violation -> "nodeRole".equals(violation.getClause().tag.name)));
     assertTrue(violations.stream().anyMatch(violation -> (violation.getClause().replica.op == Operand.LESS_THAN && "node".equals(violation.getClause().tag.name))));
 
     Policy.Suggester suggester = session.getSuggester(ADDREPLICA)
@@ -214,8 +215,64 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
 
   }
-  
-  public void testMoveReplica(){
+
+  public void testGreedyConditions() {
+    String autoscaleJson = "{" +
+        "      'cluster-policy':[" +
+        "      {'cores':'<10','node':'#ANY'}," +
+        "      {'replica':'<3','shard':'#EACH','node':'#ANY'}," +
+        "      { 'replica': 2, 'sysprop.fs': 'ssd', 'shard': '#EACH'}," +
+        "      {'nodeRole':'overseer','replica':'0'}]," +
+        "      'cluster-preferences':[" +
+        "      {'minimize':'cores', 'precision':3}," +
+        "      {'maximize':'freedisk','precision':100}]}";
+    Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
+        "node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4}," +
+        "node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3}," +
+        "node3:{cores:7, freedisk: 262, heapUsage:7834, rack: rack2, sysprop.fs : ssd}," +
+        "node4:{cores:8, freedisk: 375, heapUsage:16900, nodeRole:overseer, rack: rack1}" +
+        "}");
+
+    Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson));
+    ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState);
+    ClusterDataProvider cdp = new ClusterDataProvider() {
+      @Override
+      public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
+        return clusterDataProvider.getNodeValues(node, tags);
+      }
+
+      @Override
+      public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+        return clusterDataProvider.getReplicaInfo(node, keys);
+      }
+
+      @Override
+      public Collection<String> getNodes() {
+        return clusterDataProvider.getNodes();
+      }
+
+      @Override
+      public String getPolicy(String coll) {
+        return null;
+      }
+    };
+    Policy.Session session = policy.createSession(cdp);
+    Policy.Suggester suggester = session.getSuggester(ADDREPLICA);
+    SolrRequest op = suggester
+        .hint(Hint.COLL, "newColl")
+        .hint(Hint.SHARD, "shard1")
+        .getOperation();
+    assertNotNull(op);
+    assertEquals("node3", op.getParams().get("node"));
+    op = suggester
+        .hint(Hint.COLL, "newColl")
+        .hint(Hint.SHARD, "shard1")
+        .getOperation();
+    assertNotNull(op);
+    assertEquals("node3", op.getParams().get("node"));
+  }
+
+  public void testMoveReplica() {
     String autoscaleJson = "{" +
         "      'cluster-policy':[" +
         "      {'cores':'<10','node':'#ANY'}," +
@@ -232,17 +289,17 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "      {'core_node2':{}}]}}}");
     Map m = (Map) Utils.getObjectByPath(replicaInfoMap, false, "127.0.0.1:60089_solr/compute_plan_action_test");
     m.put("shard1", Arrays.asList(
-         new Policy.ReplicaInfo("core_node1", "compute_plan_action_test", "shard1", Collections.emptyMap()),
-         new Policy.ReplicaInfo("core_node2", "compute_plan_action_test", "shard1", Collections.emptyMap())
-     ));
-
-    Map<String, Map<String,Object>> tagsMap = (Map) Utils.fromJSONString( "{" +
-          "      '127.0.0.1:60099_solr':{" +
-          "        'cores':0," +
-          "            'freedisk':918005641216}," +
-          "      '127.0.0.1:60089_solr':{" +
-          "        'cores':2," +
-          "            'freedisk':918005641216}}}");
+        new Policy.ReplicaInfo("core_node1", "compute_plan_action_test", "shard1", Collections.emptyMap()),
+        new Policy.ReplicaInfo("core_node2", "compute_plan_action_test", "shard1", Collections.emptyMap())
+    ));
+
+    Map<String, Map<String, Object>> tagsMap = (Map) Utils.fromJSONString("{" +
+        "      '127.0.0.1:60099_solr':{" +
+        "        'cores':0," +
+        "            'freedisk':918005641216}," +
+        "      '127.0.0.1:60089_solr':{" +
+        "        'cores':2," +
+        "            'freedisk':918005641216}}}");
 
     Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson));
     Policy.Session session = policy.createSession(new ClusterDataProvider() {
@@ -268,7 +325,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
     });
 
     Policy.Suggester suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
-        .hint(Policy.Suggester.Hint.TARGET_NODE, "127.0.0.1:60099_solr");
+        .hint(Hint.TARGET_NODE, "127.0.0.1:60099_solr");
     SolrParams op = suggester.getOperation().getParams();
     assertNotNull(op);
     session = suggester.getSession();
@@ -277,7 +334,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
     assertNotNull(op);
   }
 
-  public void testOtherTag(){
+  public void testOtherTag() {
     String rules = "{" +
         "'cluster-preferences':[" +
         "{'minimize':'cores','precision':2}," +
@@ -285,13 +342,13 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "{'minimize':'heapUsage','precision':1000}" +
         "]," +
         "'cluster-policy':[" +
-        "{'nodeRole':'!overseer','strict':false}," +
+        "{replica:0, 'nodeRole':'overseer','strict':false}," +
         "{'replica':'<1','node':'node3'}," +
         "{'replica':'<2','node':'#ANY','shard':'#EACH'}" +
         "]," +
         "'policies':{" +
         "'p1':[" +
-        "{'nodeRole':'!overseer','strict':false}," +
+        "{replica:0, 'nodeRole':'overseer','strict':false}," +
         "{'replica':'<1','node':'node3'}," +
         "{'replica':'<2','node':'#ANY','shard':'#EACH'}," +
         "{'replica':'<3','shard':'#EACH','rack':'#ANY'}" +
@@ -328,7 +385,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
         return "p1";
       }
     };
-    Policy.Session session =  policy.createSession(cdp);
+    Policy.Session session = policy.createSession(cdp);
 
     CollectionAdminRequest.AddReplica op = (CollectionAdminRequest.AddReplica) session
         .getSuggester(ADDREPLICA)
@@ -339,19 +396,19 @@ public class TestPolicy extends SolrTestCaseJ4 {
   }
 
 
-  private ClusterDataProvider getClusterDataProvider(final Map<String, Map> nodeValues, String  clusterState) {
+  private ClusterDataProvider getClusterDataProvider(final Map<String, Map> nodeValues, String clusterState) {
     return new ClusterDataProvider() {
-        @Override
-        public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
-          Map<String, Object> result = new LinkedHashMap<>();
-          tags.stream().forEach(s -> result.put(s, nodeValues.get(node).get(s)));
-          return result;
-        }
-
-        @Override
-        public Collection<String> getNodes() {
-          return nodeValues.keySet();
-        }
+      @Override
+      public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
+        Map<String, Object> result = new LinkedHashMap<>();
+        tags.stream().forEach(s -> result.put(s, nodeValues.get(node).get(s)));
+        return result;
+      }
+
+      @Override
+      public Collection<String> getNodes() {
+        return nodeValues.keySet();
+      }
 
       @Override
       public String getPolicy(String coll) {
@@ -359,15 +416,15 @@ public class TestPolicy extends SolrTestCaseJ4 {
       }
 
       @Override
-        public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
-          return getReplicaDetails(node, clusterState);
-        }
+      public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+        return getReplicaDetails(node, clusterState);
+      }
 
-      };
+    };
   }
 
   public void testMultiReplicaPlacement() {
-    String autoScaleJson ="{" +
+    String autoScaleJson = "{" +
         "  'cluster-preferences': [" +
         "    { maximize : freedisk , precision: 50}," +
         "    { minimize : cores, precision: 2}" +
@@ -387,7 +444,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "}";
 
 
-    Map<String,Map> nodeValues = (Map<String, Map>) Utils.fromJSONString( "{" +
+    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}," +
@@ -420,8 +477,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
     Map<String, List<String>> locations = PolicyHelper.getReplicaLocations(
         "newColl", (Map<String, Object>) Utils.fromJSONString(autoScaleJson),
         dataProvider, Collections.singletonMap("newColl", "policy1"), Arrays.asList("shard1", "shard2"), 3);
-    assertTrue(locations.get("shard1").containsAll(ImmutableList.of("node2","node1","node3")));
-    assertTrue(locations.get("shard2").containsAll(ImmutableList.of("node2","node1","node3")));
+    assertTrue(locations.get("shard1").containsAll(ImmutableList.of("node2", "node1", "node3")));
+    assertTrue(locations.get("shard2").containsAll(ImmutableList.of("node2", "node1", "node3")));
 
 
   }
@@ -443,7 +500,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
           if (shardVsReplicaStats == null) result.put(collName, shardVsReplicaStats = new HashMap<>());
           List<Policy.ReplicaInfo> replicaInfos = shardVsReplicaStats.get(shard);
           if (replicaInfos == null) shardVsReplicaStats.put(shard, replicaInfos = new ArrayList<>());
-          replicaInfos.add(new Policy.ReplicaInfo(replicaName,collName, shard, new HashMap<>()));
+          replicaInfos.add(new Policy.ReplicaInfo(replicaName, collName, shard, new HashMap<>()));
         });
       });
     });