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

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

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();
     }
   }
 }