You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@falcon.apache.org by pr...@apache.org on 2016/03/07 07:15:55 UTC

falcon git commit: FALCON-1839 Test case for APIs for entities scheduled on native scheduler

Repository: falcon
Updated Branches:
  refs/heads/master f40926669 -> 7eb362a5e


FALCON-1839 Test case for APIs for entities scheduled on native scheduler

Author: Pragya <mi...@gmail.com>

Reviewers: "Murali Ramasami <mu...@gmail.com>, Paul Isaychuk <pi...@apache.org>, Deepak Kumar Barr <de...@gmail.com>"

Closes #49 from pragya-mittal/native-api and squashes the following commits:

a7baab5 [Pragya] Modified Changes.txt
c800405 [Pragya] Addressed review comments
4fd3cf8 [Pragya] Making waitTillInstanceReachState generic
9009d39 [Pragya] Adding assertion
8b95448 [Pragya] Adding doc for NativeInstanceUtil.waitTillInstancesAreCreated
d244279 [Pragya] Modifying NativeInstanceUtil.waitTillInstancesAreCreated based on review comments
9a882df [Pragya] Modifying totalMinutesToWait  in NativeInstancEUtil to wait for defined time
97a155f [Pragya] Review comments addressed
bbf551a [Pragya] Review commenst addressed
b674558 [Pragya] FALCON-1839 Test case for APIs for entities scheduled on native scheduler
26c462a [Pragya] Merge branch 'master' of https://github.com/apache/falcon
cd0664a [Pragya] Revert "FALCON-1832 Adding tags in SchedulableEntityInstance.compareTo()"
db094ea [Pragya] FALCON-1832 Adding tags in SchedulableEntityInstance.compareTo()
3c302e2 [Pragya] Merge branch 'master' of https://github.com/apache/falcon
f037385 [Pragya] Merge branch 'master' of https://github.com/apache/falcon
4c19ec0 [Pragya] Merge branch 'master' of https://github.com/apache/falcon
3b7fd63 [Pragya] FALCON-1829 Add regression for submit and schedule process on native scheduler (time based)


Project: http://git-wip-us.apache.org/repos/asf/falcon/repo
Commit: http://git-wip-us.apache.org/repos/asf/falcon/commit/7eb362a5
Tree: http://git-wip-us.apache.org/repos/asf/falcon/tree/7eb362a5
Diff: http://git-wip-us.apache.org/repos/asf/falcon/diff/7eb362a5

Branch: refs/heads/master
Commit: 7eb362a5ed003051723fd47357d57383f2539a1a
Parents: f409266
Author: Pragya <mi...@gmail.com>
Authored: Mon Mar 7 11:43:05 2016 +0530
Committer: Pragya Mittal <mi...@gmail.com>
Committed: Mon Mar 7 11:43:05 2016 +0530

----------------------------------------------------------------------
 falcon-regression/CHANGES.txt                   |   2 +
 .../regression/core/util/InstanceUtil.java      |   2 +-
 .../core/util/NativeInstanceUtil.java           | 184 +++++++++++
 .../nativeScheduler/NativeSchedulerAPITest.java | 307 +++++++++++++++++++
 4 files changed, 494 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/falcon/blob/7eb362a5/falcon-regression/CHANGES.txt
----------------------------------------------------------------------
diff --git a/falcon-regression/CHANGES.txt b/falcon-regression/CHANGES.txt
index 0fff007..bc76009 100644
--- a/falcon-regression/CHANGES.txt
+++ b/falcon-regression/CHANGES.txt
@@ -5,6 +5,8 @@ Trunk (Unreleased)
   INCOMPATIBLE CHANGES
 
   NEW FEATURES
+   FALCON-1839 Test case for APIs for entities scheduled on native scheduler (Pragya Mittal) 
+  
    FALCON-1566 Add test for SLA monitoring API (Pragya Mittal)
 
    FALCON-1567 Test case for Lifecycle feature  (Pragya Mittal)

http://git-wip-us.apache.org/repos/asf/falcon/blob/7eb362a5/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/InstanceUtil.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/InstanceUtil.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/InstanceUtil.java
index b07e275..c319183 100644
--- a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/InstanceUtil.java
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/InstanceUtil.java
@@ -657,7 +657,7 @@ public final class InstanceUtil {
      * @param expectedStatus expected status we are waiting for
      * @return minutes to wait for expected status
      */
-    private static int getMinutesToWait(EntityType entityType, CoordinatorAction.Status expectedStatus) {
+    public static int getMinutesToWait(EntityType entityType, CoordinatorAction.Status expectedStatus) {
         switch (expectedStatus) {
         case RUNNING:
             if (entityType == EntityType.PROCESS) {

http://git-wip-us.apache.org/repos/asf/falcon/blob/7eb362a5/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/NativeInstanceUtil.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/NativeInstanceUtil.java b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/NativeInstanceUtil.java
new file mode 100644
index 0000000..3add7a0
--- /dev/null
+++ b/falcon-regression/merlin-core/src/main/java/org/apache/falcon/regression/core/util/NativeInstanceUtil.java
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.falcon.regression.core.util;
+
+import org.apache.falcon.entity.v0.Entity;
+import org.apache.falcon.entity.v0.EntityType;
+import org.apache.falcon.entity.v0.Frequency;
+import org.apache.falcon.regression.core.helpers.ColoHelper;
+import org.apache.falcon.resource.InstancesResult;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.log4j.Logger;
+import org.apache.oozie.client.CoordinatorAction;
+import org.testng.Assert;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+/**
+ * Util functions related to native instance test.
+ * In case of entity scheduled using native scheduler, we cannot use oozie coordinator to retrieve relevant information
+ * This util retrieves instance related information using falcon APIs
+ */
+public final class NativeInstanceUtil {
+
+    public static final int INSTANCES_CREATED_TIMEOUT = OSUtil.IS_WINDOWS ? 20 : 10;
+    private static final Logger LOGGER = Logger.getLogger(NativeInstanceUtil.class);
+
+    private NativeInstanceUtil() {
+        throw new AssertionError("Instantiating utility class...");
+    }
+
+    /**
+     * Waits till instance of specific entity will be created during timeout.
+     * Timeout is common for most of usual test cases.
+     * Using status API to retrieve instance information.
+     *
+     * @param cluster     ColoHelper - colo on which API to be executed
+     * @param entity      definition of entity which describes job
+     * @param startTime   start time of instance
+     * @param endTime     end time of instance
+     */
+    public static void waitTillInstancesAreCreated(ColoHelper cluster, Entity entity, String startTime, String endTime)
+        throws InterruptedException, IOException, AuthenticationException,
+            URISyntaxException {
+        waitTillInstancesAreCreated(cluster, entity, startTime, endTime, INSTANCES_CREATED_TIMEOUT);
+    }
+
+    /**
+     * Waits till instances of specific job will be created during specific time.
+     * Use this method directly in unusual test cases where timeouts are different from trivial.
+     * In other cases use waitTillInstancesAreCreated(ColoHelper,Entity,String,String)
+     * Using status API to retrieve instance information.
+     *
+     * @param cluster     ColoHelper - colo on which API to be executed
+     * @param entity      definition of entity which describes job
+     * @param startTime   start time of instance
+     * @param endTime     end time of instance
+     * @param totalMinutesToWait     total time(in minutes) to wait for instance creation
+     */
+    public static void waitTillInstancesAreCreated(ColoHelper cluster, Entity entity, String startTime,
+        String endTime, int totalMinutesToWait)
+        throws URISyntaxException, AuthenticationException, InterruptedException, IOException {
+        String entityName = entity.getName();
+        EntityType type = entity.getEntityType();
+        String params = "?start=" + startTime;
+        params += (endTime.isEmpty() ? "" : "&end=" + endTime);
+
+        // totalSecondsToWait (totalMinutesToWait in seconds)
+        int totalSecondsToWait = totalMinutesToWait * 60;
+
+        //Incrementing sleepSeconds by 10 so that looping happens (totalMinutesToWait*60)/10 times
+        //since TimeUtil.sleepSeconds is 10
+        for (int sleepSeconds = 0; sleepSeconds < totalSecondsToWait; sleepSeconds = sleepSeconds+10) {
+            InstancesResult statusResult = cluster.getProcessHelper().getProcessInstanceStatus(entityName, params);
+            if (statusResult.getInstances() != null) {
+                return;
+            }
+            LOGGER.info(type + " " + entityName + " still doesn't have instance created");
+            TimeUtil.sleepSeconds(10);
+        }
+        Assert.fail("Instances not created");
+    }
+
+    /**
+     * Waits till given instance of process/feed reach expected state during specific time.
+     * Using status API to retrieve instance information.
+     *
+     * @param cluster           ColoHelper - colo on which API to be executed
+     * @param entity            definition of entity which describes job
+     * @param instanceTime      time of instance
+     * @param expectedStatus    expected status we are waiting for
+     * @param frequency         frequency of process/feed
+     */
+    public static void waitTillInstanceReachState(ColoHelper cluster, Entity entity, String instanceTime,
+        CoordinatorAction.Status expectedStatus, Frequency frequency)
+        throws InterruptedException, IOException, AuthenticationException, URISyntaxException {
+        int totalMinutesToWait = InstanceUtil.getMinutesToWait(entity.getEntityType(), expectedStatus);
+        waitTillInstanceReachState(cluster, entity, instanceTime, expectedStatus, frequency, totalMinutesToWait);
+    }
+
+    /**
+     * Waits till given instance of process/feed reach expected state during specific time.
+     * Using status API to retrieve instance information.
+     *
+     * @param cluster           ColoHelper - colo on which API to be executed
+     * @param entity            definition of entity which describes job
+     * @param instanceTime      time of instance
+     * @param expectedStatus    expected status we are waiting for
+     * @param frequency         frequency of process/feed
+     * @param totalMinutesToWait     total time(in minutes) to wait
+     */
+    public static void waitTillInstanceReachState(ColoHelper cluster, Entity entity, String instanceTime,
+        CoordinatorAction.Status expectedStatus, Frequency frequency, int totalMinutesToWait)
+        throws URISyntaxException, AuthenticationException, InterruptedException, IOException {
+        String entityName = entity.getName();
+        EntityType type = entity.getEntityType();
+
+        String endTime=getNextInstanceTime(instanceTime, frequency);
+        String params = "?start=" + instanceTime + "&end=" + endTime;
+
+        int totalSleepTime = totalMinutesToWait * 60;
+        int sleepTime = 10;
+        for (int i = 0; i < totalSleepTime; i = i+sleepTime) {
+            InstancesResult statusResult = cluster.getProcessHelper().getProcessInstanceStatus(entityName, params);
+            if (statusResult.getInstances() != null) {
+                if (statusResult.getInstances()[0].getStatus().name() == expectedStatus.name()) {
+                    return;
+                }
+            }
+            LOGGER.info(type + " " + entityName + " still doesn't have expected status");
+            TimeUtil.sleepSeconds(sleepTime);
+        }
+        Assert.fail("expected state of instance was never reached");
+    }
+
+    /**
+     * Returns the time of next instance for a given instanceTime.
+     *
+     * @param instanceTime     time of instance
+     * @param frequency        frequency of process/feed
+     */
+    public static String  getNextInstanceTime(String instanceTime, Frequency frequency) {
+        String nextInstanceTime;
+        int minsToAdd = 1;
+        Frequency.TimeUnit timeUnit = frequency.getTimeUnit();
+
+        switch (timeUnit) {
+        case minutes:
+            minsToAdd = frequency.getFrequencyAsInt();
+            break;
+        case hours:
+            minsToAdd = frequency.getFrequencyAsInt()*60;
+            break;
+        case days:
+            minsToAdd = frequency.getFrequencyAsInt()*60*24;
+            break;
+        case months:
+            minsToAdd = frequency.getFrequencyAsInt()*60*24*30;
+            break;
+        default:
+            Assert.fail("Unexpected freqType = " + frequency);
+            break;
+        }
+        nextInstanceTime = TimeUtil.addMinsToTime(instanceTime, minsToAdd);
+        return nextInstanceTime;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/falcon/blob/7eb362a5/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/nativeScheduler/NativeSchedulerAPITest.java
----------------------------------------------------------------------
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/nativeScheduler/NativeSchedulerAPITest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/nativeScheduler/NativeSchedulerAPITest.java
new file mode 100644
index 0000000..55ea2fb
--- /dev/null
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/nativeScheduler/NativeSchedulerAPITest.java
@@ -0,0 +1,307 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.falcon.regression.nativeScheduler;
+import org.apache.falcon.entity.v0.Frequency;
+import org.apache.falcon.regression.Entities.ProcessMerlin;
+import org.apache.falcon.regression.core.bundle.Bundle;
+import org.apache.falcon.regression.core.helpers.ColoHelper;
+import org.apache.falcon.regression.core.response.ServiceResponse;
+import org.apache.falcon.regression.core.util.AssertUtil;
+import org.apache.falcon.regression.core.util.BundleUtil;
+import org.apache.falcon.regression.core.util.OSUtil;
+import org.apache.falcon.regression.core.util.TimeUtil;
+import org.apache.falcon.regression.core.util.XmlUtil;
+import org.apache.falcon.regression.core.util.Util;
+import org.apache.falcon.regression.core.util.InstanceUtil;
+import org.apache.falcon.regression.core.util.NativeInstanceUtil;
+import org.apache.falcon.regression.testHelper.BaseTestClass;
+import org.apache.falcon.resource.EntityList;
+import org.apache.falcon.resource.EntitySummaryResult;
+import org.apache.falcon.resource.InstanceDependencyResult;
+import org.apache.falcon.resource.InstancesResult;
+import org.apache.falcon.resource.TriageResult;
+import org.apache.oozie.client.CoordinatorAction;
+import org.apache.log4j.Logger;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+/**
+ * Schedule process via native scheduler and test available APIs.
+ */
+
+@Test(groups = {"distributed", "embedded" })
+public class NativeSchedulerAPITest extends BaseTestClass {
+    private ColoHelper cluster1 = servers.get(0);
+    private String baseTestHDFSDir = cleanAndGetTestDir();
+    private String aggregateWorkflowDir = baseTestHDFSDir + "/aggregator";
+    private static final Logger LOGGER = Logger.getLogger(NativeSchedulerAPITest.class);
+    private ProcessMerlin processMerlin;
+    private String startTime;
+    private String endTime;
+
+    @BeforeClass(alwaysRun = true)
+    public void uploadWorkflow() throws Exception {
+        uploadDirToClusters(aggregateWorkflowDir, OSUtil.concat(OSUtil.RESOURCES, "sleep"));
+    }
+
+    @BeforeMethod(alwaysRun = true)
+    public void setUp() throws Exception {
+        startTime = TimeUtil.getTimeWrtSystemTime(-10);
+        endTime = TimeUtil.addMinsToTime(startTime, 50);
+        LOGGER.info("Time range between : " + startTime + " and " + endTime);
+        Bundle bundle = BundleUtil.readELBundle();
+
+        bundles[0] = new Bundle(bundle, servers.get(0));
+        bundles[0].generateUniqueBundle(this);
+        bundles[0].setProcessWorkflow(aggregateWorkflowDir);
+        bundles[0].submitClusters(prism);
+        bundles[0].setProcessConcurrency(2);
+        bundles[0].setProcessValidity(startTime, endTime);
+        bundles[0].setProcessPeriodicity(1, Frequency.TimeUnit.minutes);
+
+        //Setting input and output feed as null (time based scheduling)
+        processMerlin = bundles[0].getProcessObject();
+        processMerlin.setInputs(null);
+        processMerlin.setOutputs(null);
+        LOGGER.info(processMerlin.toString());
+
+        // Submit process
+        ServiceResponse response = prism.getProcessHelper().submitEntity(processMerlin.toString());
+        AssertUtil.assertSucceeded(response);
+
+        // Schedule with prism
+        response = prism.getProcessHelper().schedule(processMerlin.toString(), null,
+                "properties=falcon.scheduler:native");
+        AssertUtil.assertSucceeded(response);
+
+        // Schedule with server
+        response = cluster1.getProcessHelper().schedule(processMerlin.toString(), null,
+                "properties=falcon.scheduler:native");
+        AssertUtil.assertSucceeded(response);
+
+    }
+
+    @AfterMethod(alwaysRun = true)
+    public void tearDown() {
+        removeTestClassEntities();
+    }
+
+    /*Suspend and resume entity and check that it is successful.
+     */
+    @Test
+    public void entitySuspendResumeTest() throws Exception {
+        //Suspend entity via prism
+        AssertUtil.assertSucceeded(prism.getProcessHelper().suspend(processMerlin.toString()));
+
+        Assert.assertTrue(Util.parseResponse(prism.getProcessHelper()
+                .getStatus(processMerlin.toString())).getMessage()
+                .contains("SUSPENDED"));
+
+        //Resume entity via prism
+        AssertUtil.assertSucceeded(prism.getProcessHelper().resume(processMerlin.toString()));
+
+        Assert.assertTrue(Util.parseResponse(prism.getProcessHelper()
+                .getStatus(processMerlin.toString())).getMessage()
+                .contains("RUNNING"));
+
+        //Suspend entity via server
+        AssertUtil.assertSucceeded(cluster1.getProcessHelper().suspend(processMerlin.toString()));
+
+        Assert.assertTrue(Util.parseResponse(cluster1.getProcessHelper()
+                .getStatus(processMerlin.toString())).getMessage()
+                .contains("SUSPENDED"));
+
+        //Resume entity via server
+        AssertUtil.assertSucceeded(cluster1.getProcessHelper().resume(processMerlin.toString()));
+
+        Assert.assertTrue(Util.parseResponse(cluster1.getProcessHelper()
+                .getStatus(processMerlin.toString())).getMessage()
+                .contains("RUNNING"));
+    }
+
+    /*Test for entity definition, list, dependency, update and summary API and check that it is successful.
+     */
+    @Test
+    public void entityDefinitionListDependencyUpdateSummaryTest() throws Exception {
+        // Entity Definition
+        String processDef = prism.getProcessHelper().getEntityDefinition(processMerlin.toString()).getMessage();
+        Assert.assertTrue(XmlUtil.isIdentical(processMerlin.toString(), processDef), "Definitions are not equal.");
+
+        // Entity List
+        EntityList.EntityElement[] entityList = prism.getProcessHelper().listAllEntities().getEntityList().getElements();
+        Assert.assertTrue(entityList.length==1);
+        Assert.assertTrue(entityList[0].type.equals("PROCESS"));
+        Assert.assertTrue(entityList[0].name.equals(processMerlin.getName()));
+
+        // Entity Dependency
+        EntityList.EntityElement[] entityDependency = prism.getProcessHelper().getDependencies(
+                processMerlin.getName()).getEntityList().getElements();
+        Assert.assertTrue(entityDependency.length==1);
+        Assert.assertTrue(entityDependency[0].type.equals("cluster"));
+        Assert.assertTrue(entityDependency[0].name.equals(bundles[0].getClusterElement().getName()));
+
+        // Entity Update
+        NativeInstanceUtil.waitTillInstanceReachState(prism, processMerlin, startTime, CoordinatorAction.Status.RUNNING,
+                processMerlin.getFrequency());
+        processMerlin.setParallel(3);
+        LOGGER.info("Updated process xml: " + processMerlin.toString());
+        AssertUtil.assertSucceeded(prism.getProcessHelper().update(processMerlin.toString(), processMerlin.toString()));
+        processDef = prism.getProcessHelper().getEntityDefinition(processMerlin.toString()).getMessage();
+        Assert.assertTrue(XmlUtil.isIdentical(processMerlin.toString(), processDef), "Definitions are not equal.");
+
+        // Entity summary
+        EntitySummaryResult.EntitySummary[] summaries = cluster1.getProcessHelper()
+                .getEntitySummary(processMerlin.getClusterNames().get(0), "filterBy=STATUS:RUNNING")
+                .getEntitySummaryResult().getEntitySummaries();
+        Assert.assertEquals(summaries.length, 1, "There should be one RUNNING process filtered.");
+        Assert.assertEquals(summaries[0].getName(), processMerlin.getName(),
+                "Summary shows invalid suspended process.");
+    }
+
+    /*Test for instance dependency, list, params, logs, running and triage API and check that it is successful.
+     */
+    @Test
+    public void instanceAPITest() throws Exception {
+        // Instance dependency
+        NativeInstanceUtil.waitTillInstanceReachState(prism, processMerlin, startTime, CoordinatorAction.Status.RUNNING,
+                processMerlin.getFrequency());
+        InstanceDependencyResult dependencyResult = prism.getProcessHelper().getInstanceDependencies(
+                processMerlin.getName(),  "?instanceTime=" + startTime, null);
+        AssertUtil.assertSucceeded(dependencyResult);
+        Assert.assertNull(dependencyResult.getDependencies());
+
+        // Instance List
+        NativeInstanceUtil.waitTillInstanceReachState(prism, processMerlin, TimeUtil.addMinsToTime(startTime, 2),
+                CoordinatorAction.Status.SUCCEEDED, processMerlin.getFrequency());
+        InstancesResult instanceResult = prism.getProcessHelper().listInstances(processMerlin.getName(),
+                "start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 2), null);
+        InstanceUtil.validateResponse(instanceResult, 2, 0, 0, 0, 0);
+
+        // Instance Params
+        InstancesResult paramsResult = prism.getProcessHelper()
+                .getInstanceParams(processMerlin.getName(), "?start=" + startTime);
+        AssertUtil.assertSucceeded(paramsResult);
+        Assert.assertEquals(paramsResult.getInstances().length, 1);
+        Assert.assertEquals(paramsResult.getInstances()[0].getInstance(), startTime);
+
+        // Instance logs
+        InstancesResult logsResult = prism.getProcessHelper()
+                .getProcessInstanceLogs(processMerlin.getName(),
+                        "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 1));
+        AssertUtil.assertSucceeded(logsResult);
+        Assert.assertEquals(logsResult.getInstances().length, 1);
+        Assert.assertEquals(logsResult.getInstances()[0].getInstance(), startTime);
+
+        // Instance running
+        InstancesResult runningInstance = prism.getProcessHelper().getRunningInstance(processMerlin.getName());
+        InstanceUtil.validateResponse(runningInstance, 2, 2, 0, 0, 0);
+
+        // Instance triage
+        TriageResult responseTriage = prism.getProcessHelper().getInstanceTriage(processMerlin.getName(),
+                "?start=" + startTime);
+        AssertUtil.assertSucceeded(responseTriage);
+        Assert.assertEquals(responseTriage.getTriageGraphs().length, 1);
+        //There'll be just one process instance vertex and no edges
+        Assert.assertEquals(responseTriage.getTriageGraphs()[0].getVertices().length, 1);
+    }
+
+    /*Test for instance suspend, status and resume API and check that it is successful.
+     */
+    @Test
+    public void instanceSuspendResumeAPITest() throws Exception {
+
+        String range = "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 5);
+        NativeInstanceUtil.waitTillInstanceReachState(prism, processMerlin, startTime, CoordinatorAction.Status.RUNNING,
+                processMerlin.getFrequency());
+        // Suspend instance
+        InstancesResult instancesResult = prism.getProcessHelper().getProcessInstanceSuspend(
+                processMerlin.getName(), range);
+        InstanceUtil.validateResponse(instancesResult, 5, 0, 5, 0, 0);
+
+        // Suspend again
+        instancesResult = prism.getProcessHelper().getProcessInstanceSuspend(
+                processMerlin.getName(), range);
+        Assert.assertNull(instancesResult.getInstances());
+
+        // Validate instance status
+        instancesResult = prism.getProcessHelper().getProcessInstanceStatus(processMerlin.getName(),
+                range);
+        InstanceUtil.validateResponse(instancesResult, 5, 0, 5, 0, 0);
+
+        // Resume instance
+        instancesResult = prism.getProcessHelper().getProcessInstanceResume(processMerlin.getName(),
+                range);
+        InstanceUtil.validateResponse(instancesResult, 5, 2, 0, 0, 0);
+
+        // Resume again
+        instancesResult = prism.getProcessHelper().getProcessInstanceResume(processMerlin.getName(),
+                range);
+        Assert.assertNull(instancesResult.getInstances());
+
+        // Validate instance status
+        instancesResult = prism.getProcessHelper().getProcessInstanceStatus(processMerlin.getName(),
+                range);
+        InstanceUtil.validateResponse(instancesResult, 5, 2, 0, 0, 0);
+    }
+
+    /*Test for instance kill and rerun and check that it is successful.
+     */
+    @Test
+    public void instanceKillRerunAPITest() throws Exception {
+        String range = "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 5);
+        NativeInstanceUtil.waitTillInstanceReachState(prism, processMerlin, startTime, CoordinatorAction.Status.RUNNING,
+                processMerlin.getFrequency());
+
+        //Instance Kill
+        InstancesResult instancesResult = prism.getProcessHelper().getProcessInstanceKill(processMerlin.getName(),
+                range);
+        InstanceUtil.validateResponse(instancesResult, 5, 0, 0, 0, 5);
+
+        // Instance rerun
+        prism.getProcessHelper().getProcessInstanceRerun(processMerlin.getName(),
+                "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 3));
+        NativeInstanceUtil.waitTillInstanceReachState(prism, processMerlin, startTime, CoordinatorAction.Status.RUNNING,
+                processMerlin.getFrequency());
+
+        instancesResult = prism.getProcessHelper().getProcessInstanceStatus(processMerlin.getName(), range);
+        InstanceUtil.validateResponse(instancesResult, 5, 2, 0, 0, 3);
+
+    }
+
+    /*Test for rerun of succeeded instance and check that it is successful.
+     */
+    @Test
+    public void instanceSucceedRerunAPITest() throws Exception {
+        String range = "?start=" + startTime + "&end=" + TimeUtil.addMinsToTime(startTime, 3);
+        NativeInstanceUtil.waitTillInstanceReachState(prism, processMerlin, TimeUtil.addMinsToTime(startTime, 3),
+                CoordinatorAction.Status.SUCCEEDED, processMerlin.getFrequency());
+
+        // Rerun succeeded instance
+        prism.getProcessHelper().getProcessInstanceRerun(processMerlin.getName(), range + "&force=true");
+        NativeInstanceUtil.waitTillInstanceReachState(prism, processMerlin, startTime, CoordinatorAction.Status.RUNNING,
+                processMerlin.getFrequency());
+
+        InstancesResult instanceResult = prism.getProcessHelper().getProcessInstanceStatus(
+                processMerlin.getName(), range);
+        InstanceUtil.validateResponse(instanceResult, 3, 2, 0, 0, 0);
+    }
+
+}